You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "satishd (via GitHub)" <gi...@apache.org> on 2023/04/13 11:00:23 UTC

[GitHub] [kafka] satishd opened a new pull request, #13561: [DRAFT] Added remote log segments retention functionality based on time and size.

satishd opened a new pull request, #13561:
URL: https://github.com/apache/kafka/pull/13561

   This is a draft version and is not yet ready for review.
   
    - RLMTask runs retention cleanup activity in each execution for the targeted topic partition. It checks for total size, retention time and deletes the eligible segments from remote storage. It also updates the log-start-offset in UnifiedLog respectively.
   
   TODO: add tests
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1597034800

   > @satishd , any update for this PR? If you don't have time on it, just let me know. :)
   
   addressing the review comments in progress, needs minor refactoring which is going on. Will have those changes pushed in the next couple of days. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213107956


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the

Review Comment:
   Updated the comment to make it more clear.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1283024088


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   It is not mandatory to update it when this node becomes a follower as the existing follower fetch protocol makes sure that the follower truncates their log-start-offset based on the leader's log-start-ffset. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293081560


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   In the FETCH response, the leader-log-start-offset will be piggy-backed. But, there can be a scenario:
   
   1. Leader deleted the remote log segment and updates it's log-start-offset
   2. Before the replica-2 update it's log-start-offset via FETCH-request, the leadership changed to replica-2.
   3. There are no more eligible segments to delete from remote.
   4. The log-start-offset will be stale (referring to old log-start-offset but the data was already removed from remote)
   5. If the consumer starts to read from the beginning of the topic, it will fail to read.
   
    I realised the case mentioned by @divijvaidya and this one is different. Both of them can be handled by the new leader gracefully. We can take this task in a follow-up PR if required.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293851913


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,
+                                                            long logEndOffset,
+                                                            NavigableMap<Integer, Long> leaderEpochs) {

Review Comment:
   We want to be a bit careful of using this method. LeaderEpochCache is mostly derived from the data in the log. However, on new leader epoch from a leader change, the new leader also appends the new epoch to LeaderEpochCache before any record is appended for the epoch. This could cause a slight mis-match between the epoch chain in the remote segment and LeaderEpochCache. For example, it's possible for a LeaderEpochCache to have
   
   10 100
   11  200 //no record appended for epoch 11
   12 200
   
   where a segment's epoch chain only has 
   10 100
   12 200
   
   We don't want to prevent the remote segment from being deleted through the retention logic because of this slight mismatch on leader epoch chain. Does the code allow for this?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1180121259


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();
+                } else {
+                    Optional<UnifiedLog> unifiedLogOptional = fetchLog.apply(topicIdPartition.topicPartition());
+                    if (unifiedLogOptional.isPresent()) {
+                        long offset = findHighestRemoteOffset(topicIdPartition);
+                        unifiedLogOptional.get().updateHighestOffsetInRemoteStorage(offset);

Review Comment:
   We might need to add logs here to describe why we need to update highest offset in remote storage for followers. I think that's for fetch from follower replica feature, right?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0

Review Comment:
   nit: assume that segments contain size >= 0



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);

Review Comment:
   I agree this could be costly if the RLMM implementation doesn't cache the metadata. But I don't think there's an implementation constraint for plugin providers. They can always cache them in the plugin. I'm thinking it should be enough if we add something about it in the RLMM#listRemoteLogSegments javadoc.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   > Should this operation be performed in a separate thread pool which can have a defined quota? (similar to how we perform cleaning for local log using separate cleaner/background threads).
   
   Good suggestion. But we don't include this part in the original KIP, we need another KIP to improve 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dopuskh3 commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "dopuskh3 (via GitHub)" <gi...@apache.org>.
dopuskh3 commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1307788374


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            // Build the leader epoch map by filtering the epochs that do not have any records.
+            NavigableMap<Integer, Long> epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets());
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            long logStartOffset = log.logStartOffset();
+            long logEndOffset = log.logEndOffset();
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), logEndOffset, epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, logStartOffset);
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());

Review Comment:
   @satishd it seems I'm reaching that codepath when running reassignments on my cluster and segment are deleted from remote store despite a huge retention (topic created a few hours ago with 1000h retention). 
   It seems to happen consistently on some partitions when reassigning but not all partitions.
   
   
   My test: 
   
   I have a test topic with 30 partition configured with 1000h global retention and 2 minutes local retention
   I have a load tester producing to all partitions evenly
   I have consumer load tester consuming that topic
   I regularly reset offsets to earliest on my consumer to test backfilling from tiered storage.
   
   My consumer was catching up consuming the backlog and I wanted to upscale my cluster to speed up recovery: I upscaled my cluster from 3 to 12 brokers and reassigned my test topic to all available brokers to have an even leader/follower count per broker.
   
   When I triggered the reassignment, the consumer lag dropped on some of my topic partitions: 
   <img width="2426" alt="Screenshot 2023-08-28 at 20 57 09" src="https://github.com/apache/kafka/assets/86608/14e8f43d-da77-40b9-83c6-bb930fe2763d">
   
   Later I tried to reassign back my topic to 3 brokers and the issue happened again. 
   
   Both times in my logs, I've seen a bunch of logs like:
   ```
   [RemoteLogManager=10005 partition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17] Deleted remote log segment RemoteLogSegmentId{topicIdPartition=uR3O_hk3QRqsn4mPXGFoOw:loadtest11-17, id=Mk0chBQrTyKETTawIulQog} due to leader epoch cache truncation. Current earliest epoch: EpochEntry(epoch=14, startOffset=46776780), segmentEndOffset: 46437796 and segmentEpochs: [10]
   ```
   
   Looking at my s3 bucket. The segments prior to my reassignment have been indeed deleted.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1284280124


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -667,11 +675,323 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");

Review Comment:
   Replied in the [comment](https://github.com/apache/kafka/pull/13561#discussion_r1283024088).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294618085


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1033,6 +1360,35 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < remainingBreachedSize) {
+                throw new IllegalArgumentException("retentionSize must be greater than remainingBreachedSize");
+            }

Review Comment:
   Could we elaborate what's the purpose of this validation? IIUC `(totalSize - retentionSize) > retentionSize`, are we validating that totalSize is not higher than 2 times `retentionSize`?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1296221075


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Yes, I agree that it's not a large and common issue. So, we can leave it as it is for now.



##########
core/src/main/scala/kafka/server/KafkaServer.scala:
##########
@@ -614,7 +614,13 @@ class KafkaServer(
       }
 
       Some(new RemoteLogManager(config.remoteLogManagerConfig, config.brokerId, config.logDirs.head, clusterId, time,
-        (tp: TopicPartition) => logManager.getLog(tp).asJava, brokerTopicStats));
+        (tp: TopicPartition) => logManager.getLog(tp).asJava,
+        (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
+          logManager.getLog(tp).foreach(log => {

Review Comment:
   This can be simplified a bit to `.foreach{ log => ...}`. Ditto for the same code in BrokerServer.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1682651711

   >Do you know why there are 100+ test failures?
   
   @junrao Those failures(except one) are not related to this PR. Updated with a few minor changes and tests. The latest run had a few failures which seem to be unrelated to this change. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1182338603


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   Hmm, I am not sure this is the right thing to do, because including segments which are not part of a log yields a size which is not truly that of the log. It is possible to design scenarios with a log chronology which allows for premature deletion of data under size-based retention.
   
   While I understand that deleting unreferenced segments is consistent with the local log use case, where some data can be lost too, a key difference here between this approach and the current retention semantics applied for local logs is that in the latter case, all segments belong to the current log, so that the size-based retention policy always apply to the current log. Eviction of unreferenced segments/data in the local case happens via truncation which happens separately from the enforcement of retention policies. But here, both are combined.
   
   What are the benefits of diverging from these semantics with tiered 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1222183119


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -945,4 +1176,27 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+
+    }
+
+    private static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            this.retentionMs = retentionMs;
+            this.cleanupUntilMs = cleanupUntilMs;
+        }
+

Review Comment:
   extra new line



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -945,4 +1176,27 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+

Review Comment:
   extra new line



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -945,4 +1176,27 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+
+    }
+
+    private static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            this.retentionMs = retentionMs;
+            this.cleanupUntilMs = cleanupUntilMs;
+        }
+
+    }
+

Review Comment:
   extra new line



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();

Review Comment:
   Let's say there is a remote segment with startOffset 100 and endOffset 200. If the localLogStartOffset is 150, we exclude the remote segment. This means that we are undercounting the size, right?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1383,19 +1430,22 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   }
 
   private def deleteRetentionMsBreachedSegments(): Int = {
-    if (config.retentionMs < 0) return 0
+    val retentionMs = localRetentionMs(config)
+    if (retentionMs < 0) return 0
     val startMs = time.milliseconds
 
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      startMs - segment.largestTimestamp > config.retentionMs
+      startMs - segment.largestTimestamp > retentionMs
     }
 
     deleteOldSegments(shouldDelete, RetentionMsBreach(this))
   }
 
+

Review Comment:
   extra new line



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        return;
+                    }
+
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known

Review Comment:
   Hmm, why do we need a separate retention based on leader epochs? Is that not already covered by size/time/startOffset based retention?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1331,7 +1370,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage

Review Comment:
   Is highestOffsetInRemoteStorage inclusive or exclusive? It would be useful to document that.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -945,13 +978,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {

Review Comment:
   Hmm, this logic doesn't look right. If a client calls `deleteRecords`, we call `maybeIncrementLogStartOffset` with onlyLocalLogStartOffsetUpdate=false. So, we will go through this branch and update _localLogStartOffset. This will be incorrect if remote log is enabled.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +

Review Comment:
   We already log the completion of the load loading in LogManager. Could we fold this there to avoid double logging?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {
+    _localLogStartOffset = offset
+
+    if (highWatermark < offset) {
+      updateHighWatermark(offset)
+    }
+
+    if (this.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)

Review Comment:
   This is already done in `updateLogStartOffset`. Do we need to do it here again?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1281568519


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   I agree with @divijvaidya 's suggestion. In this scenario:
   1. replica 1 is the leader, and doing remote log segment deletion
   2. leadership changed to replica 2
   3. replica 1 entering this `handleLogStartOffsetUpdate` method
   
   Under current implementation, we won't update log start offset since it is not the leader anymore. But we should update it! @satishd , thoughts?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -634,6 +642,241 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {

Review Comment:
   +1, or at least a debug level.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   @jeqo , What Satish meant, is in most remote storage case, the deletion API won't wait until data deleted in remote storage, instead, it'll mark file as deleted and return immediately. And run background GC in remote storage to delete the deleted flagged file.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1246792402


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -147,11 +147,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   def localLogStartOffset(): Long = _localLogStartOffset
 
+  // This is the offset(inclusive) until which segments are copied to the remote storage.
   @volatile private var highestOffsetInRemoteStorage: Long = -1L
 
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()

Review Comment:
   I also find it strange to repeat the mutations of HWM and local log recovery point in both updates, we can pull those two updates into a single method and call it once?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -945,13 +976,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          _localLogStartOffset = math.max(newLogStartOffset, localLogStartOffset())
+
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {
+            updatedLogStartOffset = true
+            updateLogStartOffset(newLogStartOffset)

Review Comment:
   is it correct to update localLogStartOffset directly, but use updateLogStartOffset method to also update related values (HWM and local log recovery)?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181545427


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -942,13 +974,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          _localLogStartOffset = math.max(newLogStartOffset, localLogStartOffset())
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {
+            updatedLogStartOffset = true
+            updateLogStartOffset(newLogStartOffset)
+            _localLogStartOffset = newLogStartOffset

Review Comment:
   It's confusing we update `_localLogStartOffset` twice with different value. I think the one in L982 should be removed, right?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1287764297


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -956,6 +981,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  private def maybeIncrementLocalLogStartOffset(newLocalLogStartOffset: Long, reason: LogStartOffsetIncrementReason): Unit = {
+    lock synchronized {
+      if (newLocalLogStartOffset > localLogStartOffset()) {
+        _localLogStartOffset = math.max(newLocalLogStartOffset, localLogStartOffset());

Review Comment:
   Since newLocalLogStartOffset is larger than localLogStartOffset(), could we just assign newLocalLogStartOffset to _localLogStartOffset?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -966,7 +1000,8 @@ class UnifiedLog(@volatile var logStartOffset: Long,
    * @throws OffsetOutOfRangeException if the log start offset is greater than the high watermark
    * @return true if the log start offset was updated; otherwise false
    */
-  def maybeIncrementLogStartOffset(newLogStartOffset: Long, reason: LogStartOffsetIncrementReason): Boolean = {
+  def maybeIncrementLogStartOffset(newLogStartOffset: Long,
+                                           reason: LogStartOffsetIncrementReason): Boolean = {

Review Comment:
   Identation doesn't match other places in this file.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Here is a corner case. Let's say remote log is enabled, but there is no remote segment (all have been deleted due to retention). The new logic will do retention based on `localRetentionBytes`, but it should actually do the retention based on `retentionSize`. If that happens, we need to advance logStartOffset, in addition to localLogStartOffset.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1464,12 +1513,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled()))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
+      nextSegmentOpt.exists(_.baseOffset <= localLogStartOffset())

Review Comment:
   This doesn't look right. If remote log is not enabled, it seems that we should delete based on logStartOffset, not localLogStartOffset.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage
+        else true

Review Comment:
   Hmm, this should be false, right? Do we have a test case to cover that?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -147,11 +147,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
   def localLogStartOffset(): Long = _localLogStartOffset
 
+  // This is the offset(inclusive) until which segments are copied to the remote storage.
   @volatile private var highestOffsetInRemoteStorage: Long = -1L
 
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))

Review Comment:
   This is an existing issue. But there is one direct reference to `_localLogStartOffset` in `fetchOffsetByTimestamp()`. Should we change that to use `localLogStartOffset()` instead?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   > All the size/time/startOffset handlers run based on the current leader’s leader epochs. Here, we are removing the segments which have leader epochs earlier to the lowest leader epoch on this broker(partition leader).
   
   Hmm, I still don't quite understand this part. The leader's epoch chain only gets trimmed from the beginning when segments are deleted due to retention or the advancement of the startOffset by `deleteRecord()` call. These are covered by the size/time based retention and logStartOffset based retention. So what additional cases does the following code cover?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   > All the size/time/startOffset handlers run based on the current leader’s leader epochs. Here, we are removing the segments which have leader epochs earlier to the lowest leader epoch on this broker(partition leader).
   
   Hmm, I still don't quite understand this part. The leader's epoch chain only gets trimmed from the beginning when segments are deleted due to retention or the advancement of the startOffset by `deleteRecord()` call. These are covered by the size/time based retention and logStartOffset based retention. So what additional cases does the following code cover?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1672759994

   Thanks @junrao for the review. Addressed the review comments inline and/or with the latest commits.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295394486


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   Filed KAFKA-15351 and KAFKA-15352 to track the cases.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293224397


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();
+                } else {
+                    Optional<UnifiedLog> unifiedLogOptional = fetchLog.apply(topicIdPartition.topicPartition());
+                    if (unifiedLogOptional.isPresent()) {
+                        long offset = findHighestRemoteOffset(topicIdPartition);
+                        unifiedLogOptional.get().updateHighestOffsetInRemoteStorage(offset);

Review Comment:
   @satishd can you please address this comment. Multiple folks have asked me why this code of line exists which makes me believe that a comment explaining the purpose here would be nice.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1298640349


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   What's the impact to 3.6.0? Do we need to outline any limitation with unclean leader election in the release notes?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1182338603


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   Hmm, I am not sure this is the right thing to do, because including segments which are not part of a log yields a size which is not truly that of the log. It is possible to design scenarios with a log chronology which allows for premature deletion of data under size-based retention.
   
   While I understand that deleting unreferenced segments is consistent with the local log use case, where some data can be lost too, a key difference here between this approach and the current retention semantics applied for local logs is that in the latter case, all segments belong to the current log when the log size is calculated, so that the size-based retention policy always apply to the current log. Eviction of unreferenced segments/data in the local case happens via truncation separately from the enforcement of retention policies. But here, both are retention-based and truncation-driven eviction are _de facto_ combined.
   
   What are the benefits of diverging from these semantics with tiered 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1182338603


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   Hmm, I am not sure this is the right thing to do, because including segments which are not part of a log yields a size which is not truly that of the log. It is possible to design scenarios with a log chronology which allows for premature deletion of data under size-based retention.
   
   While I understand that deleting unreferenced segments is consistent with the local log use case, where some data can be lost too, a key difference here between this approach and the current retention semantics applied for local logs is that in the latter case, all segments belong to the current log when the log size is calculated, so that the size-based retention policy always apply to the current log. Eviction of unreferenced segments/data in the local case happens via truncation separately from the enforcement of retention policies. But here, both are combined.
   
   What are the benefits of diverging from these semantics with tiered 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on pull request #13561: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1508030768

   `KAFKA-14888: RemoteLogManager - deleting expired/size breached log segments to remote storage implementation` was created for this task. I've updated the PR title, FYI


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1296731885


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   >One way to solve is it is to delete the data that we know for sure is ready for deletion, e.g. if we have 10MB of data in remote store for non-active lineage and retention size is 2MB, then we can safely delete the rest of the 8MB. This is because even if this leadership chain becomes active, it will adhere to retention size.
   
   It is hard or impossible to find the non-active lineage deterministically as the failed host can have any subset of the non active lineage. Determining which epoch/segments can be marked for deletion under such circumstances is not feasible. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300055517


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   This takes an assumption that the partition has continuous monotonically increasing offsets. But it is not true for a topic that was historically compacted (but compacted is turned off now, that is why TS is enabled).
   
   I would suggest to read the next segment and set the startOffset as the start offset of the next segment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295387916


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1033,6 +1360,35 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < remainingBreachedSize) {
+                throw new IllegalArgumentException("retentionSize must be greater than remainingBreachedSize");
+            }

Review Comment:
   Good catch! It was changed while refactoring, added UTs to cover that in the latest commits.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1692563258

   Merging it to 3.6 branch


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289488513


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage
+        else true

Review Comment:
   No, this should be true if the remote storage is not enabled as this segment should be eligible based on other checks like `highWatermark >= upperBoundOffset && predicate(segment, nextSegmentOpt)`. Existing tests in `UnifiedLogTest`, `LogOffsetTest`, `LogLoaderTest`, `LogCleanerTest` already cover those scenarios.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1308153773


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            // Build the leader epoch map by filtering the epochs that do not have any records.
+            NavigableMap<Integer, Long> epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets());
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            long logStartOffset = log.logStartOffset();
+            long logEndOffset = log.logEndOffset();
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), logEndOffset, epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, logStartOffset);
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());

Review Comment:
   Thanks @dopuskh3 for bringing the observed issue here. I will followup on [JIRA](https://issues.apache.org/jira/browse/KAFKA-15414). There are a few more pending changes to be merged which are in review/planned. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1281672320


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        return;
+                    }
+
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known

Review Comment:
   All the size/time/startOffset handlers run based on the current leader’s leader epochs. Here, we are removing the segments which have leader epochs earlier to the lowest leader epoch on this broker(partition leader).



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        return;
+                    }
+
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known

Review Comment:
   All the size/time/startOffset handlers run based on the current leader’s leader epochs. Here, we are removing the segments which have leader epochs earlier to the lowest leader epoch on this broker(partition leader).



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1597103576

   Rebased with the trunk as this PR had conflicts because of other introduced changes in the trunk.
   
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1234980098


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();

Review Comment:
   I guess that is fine as retention size is more about the minimum size available in the topic partition. That segment will be deleted when the local-log-start-offset moves in later cycles.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1234980957


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {
+    _localLogStartOffset = offset
+
+    if (highWatermark < offset) {
+      updateHighWatermark(offset)
+    }
+
+    if (this.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)

Review Comment:
   It was updated based on log-start-offset with `updateLogStartOffset`, but local-log-start-offset can be more than that and it will be updated if needed.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1575514622

   @divijvaidya I will let you know once I address the remaining few comments in the next couple of days.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1299640804


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   The remote storage retention cleanup mechanism considers cleaning up the remote log segments that have all the records that are created with a leader epoch precedes to the earliest leader epoch in the current leader's leader epoch lineage. 
   
   In case of unclean leader election, the earlier leader replica may delete the segments that are copied to remote storage but those are not part of its leader epoch lineage but they may be part of out-of-sync or offline follower replicas and they will not be available for consumption.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1299988151


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1096,6 +1493,43 @@ public void close() {
         }
     }
 
+    // Visible for testing
+    public static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < 0)
+                throw new IllegalArgumentException("retentionSize should be non negative, but it is " + retentionSize);
+
+            if (remainingBreachedSize <= 0) {
+                throw new IllegalArgumentException("remainingBreachedSize should be more than zero, but it is " + remainingBreachedSize);
+            }
+
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+    }
+
+    // Visible for testing
+    public static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            if (retentionMs < 0)
+                throw new IllegalArgumentException("retentionMs should be non negative, but it is " + retentionMs);
+
+            if (retentionMs < cleanupUntilMs) {

Review Comment:
   Similar to the previous comment on RentetionSizeData: `cleanupUntilMs` represents a point in time (`now - retentionMs`), while `retentionMs` represents a duration (e.g. 1 week in millis). Is this comparison correct/needed? If I'm reading this right, this will always be true.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1299185257


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,

Review Comment:
   Good point. Added the required filtering check `findOffsetByTimestamp` API. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1298027732


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   That is a fair point. If we want to take that approach, we should not delete any segments beyond the current leader's leader epoch lineage. We need to take the risk of RSM plugins having cleanup mechanisms for the segment leaks in the remote storage. It is hard even for RSM plugin owners to deterministically find out whether a segment is unreferenced when there are out-of-sync/offline replicas. 
   
   I filed [KAFKA-15376](https://issues.apache.org/jira/browse/KAFKA-15376) to continue the discussion and take a final call later before productionizing 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294843843


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {

Review Comment:
   This method is used only from `locally` block and it does not require taking any lock. We moved this method inside the locally block to avoid any confusion and future usage outside of that.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1296264710


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   >It is hard to know whether a particular lineage can exist in any of the replicas as replicas can fail and it is hard to say whether a particular replica can come back with in a specific duration.
   
   Correct, that is why we should not be deleting data that we are unsure about. It's a durability loss! In a trade-off situation, wouldn't we want to trade-off in favour of durability instead of remote storage leak (which can be GC'ed by RSM implementation for such cases).
   
   One way to solve is it is to delete the data that we know for sure is ready for deletion, e.g. if we have 10MB of data in remote store for non-active lineage and retention size is 2MB, then we can safely delete the rest of the 8MB. This is because even if this leadership chain becomes active, it will adhere to retention size.
   
   In other words, I am not saying that we should not delete non-active lineage data in remote store. I am saying that the non-active lineage data should only be deleted if it when it is violating the retention policies. If we have time based retention, this will ensure that there are no leaks. If we have size based retention, then we can do what you are suggesting.
   
   I will not consider this blocking to land this PR since this is in early access but we should document this risk of data loss as part of release notes and try to arrive at a conclusion before production release.
   
   Thoughts @showuon @junrao @ivanyu @jeqo ?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1308153773


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            // Build the leader epoch map by filtering the epochs that do not have any records.
+            NavigableMap<Integer, Long> epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets());
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            long logStartOffset = log.logStartOffset();
+            long logEndOffset = log.logEndOffset();
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), logEndOffset, epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, logStartOffset);
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());

Review Comment:
   Thanks @dopuskh3 for bringing the observed issue here. There are a few more pending changes to be merged which are in review/planned related to this change. I will followup on [JIRA](https://issues.apache.org/jira/browse/KAFKA-15414). 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1659937748

   @satishd , is this PR ready for another round of review?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1209013637


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,192 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention, long retentionSize) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.", 
+                            metadata.remoteLogSegmentId(), retentionSize, remainingBreachedSize + retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata, checkTimestampRetention, cleanupTs, retentionMs) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata, checkSizeRetention, log.config().retentionSize) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known

Review Comment:
   nit: less than



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1284278935


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2207,7 +2267,7 @@ case class RetentionSizeBreach(log: UnifiedLog) extends SegmentDeletionReason {
     var size = log.size
     toDelete.foreach { segment =>
       size -= segment.size
-      log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+      log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config)} breach. Local log size " +

Review Comment:
   Good catch.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293244517


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);

Review Comment:
   could we please store the value of log.logEndOffset() at the beginning of clean up process and use the stored value for all calculations? Asking because endOffset may move behind the scenes while we are processing cleaning. 
   
   The overall idea is that this cleanup should be executing on a snapshot of log state.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);

Review Comment:
   Note that same segment may span across multiple epochs. Hence, same segment ID will be returned multiple times here and we will count it's size multiple times. 
   
   May I suggest:
   
   ```
   epochEntries.navigableKeySet().iterator
       .flatMap(epochEntry => remoteLogMetadataManager.listRemoteLogSegments(tpId, epochEntry.epoch).asScala)
       .filter(isRemoteSegmentWithinLeaderEpochs(epochEntries, _, logEndOffset))
       .distinctBy(_.remoteLogSegmentId.id())
       .map(segment => segment.segmentSizeInBytes())
       .reduceOption((a, b) => a.add(b))
       .getOrElse(0)
   ```
   
   
   Also, if you agree that this was a bug, please add a unit test that should have failed.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,

Review Comment:
   you need to use this to correctly filter out segments at `findOffsetByTimestamp` method as well please.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2173,6 +2225,14 @@ object UnifiedLog extends Logging {
     }
   }
 
+  private[log] def localRetentionMs(config: LogConfig): Long = {
+    if (config.remoteLogConfig.remoteStorageEnable) config.remoteLogConfig.localRetentionMs else config.retentionMs
+  }
+
+  private[log] def localRetentionSize(config: LogConfig): Long = {
+    if (config.remoteLogConfig.remoteStorageEnable) config.remoteLogConfig.localRetentionBytes else config.retentionSize
+  }
+

Review Comment:
   You can instead use similar methods already present in LogConfig.
   
   see LogConfig.localRetentionBytes() and LogConfig.localRetentionMs()
   
   (you will probably have to modify them to add new case of `if (config.remoteLogConfig.remoteStorageEnable)`
   
   



##########
storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java:
##########
@@ -102,9 +102,9 @@ public String topicWarningMessage(String topicName) {
 
     public static class RemoteLogConfig {
 
-        private final boolean remoteStorageEnable;
-        private final long localRetentionMs;
-        private final long localRetentionBytes;
+        public final boolean remoteStorageEnable;
+        public final long localRetentionMs;
+        public final long localRetentionBytes;

Review Comment:
   I believe we already have public accessor functions in LogConfig for these.
   
   See LogConfig.localRetentionMs(), LogConfig.localRetentionBytes() and LogConfig.remoteStorageEnable()



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {

Review Comment:
   The code in this PR still uses this method. No? What am I missing?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   Isn't it possible for older epoch chain to become the current chain after another unclean election?
   
   For example:
   
   Time T1: Leader epoch chain
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        0              0
        1              180
        2              400
   -----------------------------
   ```
   Time T2: Unclean leader election occurs where the new leader loses all existing data and starts with new leader epoch
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        3              700
        4              780
        6              900
        7              990 					
   -----------------------------
   ```
   Time T3: Unclean leader election occurs again but the old leader from T1 becomes new leader (epoch 8). In this case, the current epoch chain will be 0->1->2->8. But we have deleted data from remote already pertaining to 0,1 and 2, even if it was not eligible for deletion based on retention. 
   
   To remedy this situation, may I suggest that we delete the unreferenced segments "only" if we definitely know that they can be cleaned i.e. when they have exceeded the retention time or when the size in remote itself is greater than retention size. I have to check but I believe that local log solves it in a similar manner.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293336126


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   The current code uses the leader epoch chain to calculate the size. This comment is resolved in latest code.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294603142


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};
+
+        // Test whether a remote segment's epochs/offsets(multiple) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(
+                15,
+                35,
+                new TreeMap<Integer, Long>() {{
+                    put(1, 15L);
+                    put(2, 20L);
+                    put(3, 30L);
+                }}), logEndOffset, leaderEpochToStartOffset));
+
+        // Test whether a remote segment's epochs/offsets(single) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(
+                15,
+                19,
+                new TreeMap<Integer, Long>() {{
+                    put(1, 15L);
+                }}), logEndOffset, leaderEpochToStartOffset));
+
+        // Test whether a remote segment's epochs/offsets(single) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(

Review Comment:
   Good catch!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294829317


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1033,6 +1365,35 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < remainingBreachedSize) {
+                throw new IllegalArgumentException("retentionSize must be greater than remainingBreachedSize");
+            }
+
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+    }
+
+    private static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            if (retentionMs < cleanupUntilMs) {
+                throw new IllegalArgumentException("retentionMs must be greater than cleanupUntilMs");

Review Comment:
   ```suggestion
                   throw new IllegalArgumentException("retentionMs [" + retentionMs + "] must be greater than cleanupUntilMs [" + cleanupUntilMs + "]");
   ```



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1033,6 +1365,35 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < remainingBreachedSize) {
+                throw new IllegalArgumentException("retentionSize must be greater than remainingBreachedSize");

Review Comment:
   ```suggestion
                   throw new IllegalArgumentException("retentionSize [" + retentionSize + "] must be greater than remainingBreachedSize [" + remainingBreachedSize + "]");
   
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1182338603


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   Hmm, I am not sure this is the right thing to do, because including segments which are not part of a log yields a size which is not truly that of the log. It is possible to design scenarios with a log chronology which allows for premature deletion of data under size-based retention.
   
   While I understand that deleting unreferenced segments is consistent with the local log use case, where some data can be lost too, a key difference here between this approach and the current retention semantics applied for local logs is that in the latter case, all segments belong to the current log when the log size is calculated, so that the size-based retention policy always apply to the current log. Eviction of unreferenced segments/data in the local case happens via truncation which happens separately from the enforcement of retention policies. But here, both are combined.
   
   What are the benefits of diverging from these semantics with tiered 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181529149


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each

Review Comment:
   Sure, let us discuss this out side of this PR. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181528820


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.

Review Comment:
   Unreferenced segments within the current leader epoch chain will eventually move earlier to the earliest epoch of the current leader epoch chain after a few retention checks. That will take care of those kinds of 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1301361619


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1096,6 +1493,43 @@ public void close() {
         }
     }
 
+    // Visible for testing
+    public static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < 0)
+                throw new IllegalArgumentException("retentionSize should be non negative, but it is " + retentionSize);
+
+            if (remainingBreachedSize <= 0) {
+                throw new IllegalArgumentException("remainingBreachedSize should be more than zero, but it is " + remainingBreachedSize);
+            }
+
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+    }
+
+    // Visible for testing
+    public static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            if (retentionMs < 0)
+                throw new IllegalArgumentException("retentionMs should be non negative, but it is " + retentionMs);
+
+            if (retentionMs < cleanupUntilMs) {

Review Comment:
   Right, I fixed the validation check. Thanks. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd merged pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd merged PR #13561:
URL: https://github.com/apache/kafka/pull/13561


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1685049750

   @satishd : The latest build still has 120 test failures. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1297576757


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2173,6 +2230,14 @@ object UnifiedLog extends Logging {
     }
   }
 
+  private[log] def localRetentionMs(config: LogConfig): Long = {
+    if (config.remoteLogConfig.remoteStorageEnable) config.remoteLogConfig.localRetentionMs else config.retentionMs

Review Comment:
   Hmm, should we use `remoteLogEnabled()` instead of `config.remoteLogConfig.remoteStorageEnable`? Ditto for `localRetentionSize`.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   @divijvaidya has a good point. When doing unclean leader election, the new leader (even if unclean) should still have access to the remote data. So, it probably should never lose that portion of the data?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1685264422

   >The latest build still has 120 test failures.
   
   @junrao Those tests are not related to the changes in the PR. The next [run](https://ci-builds.apache.org/blue/organizations/jenkins/Kafka%2Fkafka-pr/detail/PR-13561/51/tests/) had 
    one related test failure(PlaintextAdminIntegrationTest.testOffsetsForTimesAfterDeleteRecords) which is fixed with the latest commit.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1298027732


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   That is a fair point. If we want to take that approach, we should not delete any segments beyond the current leader's leader epoch lineage. We need to take the risk of RSM plugins having cleanup mechanisms for the segment leaks in the remote storage. It is hard even for RSM plugin owners to deterministically find out whether a segment is unreferenced when there are out-of-sync/offline replicas. 
   
   I filed [KAFKA-15376](https://issues.apache.org/jira/browse/KAFKA-15376) to continue the discussion and take a final call later before productionizing 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295670320


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};

Review Comment:
   Thanks @kamalcph for the clarification, good to know about that.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1296264710


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   >It is hard to know whether a particular lineage can exist in any of the replicas as replicas can fail and it is hard to say whether a particular replica can come back with in a specific duration.
   
   Correct, that is why we should not be deleting data that we are unsure about. It's a durability loss! In a trade-off situation, wouldn't we want to trade-off in favour of durability instead of remote storage leak (which can be GC'ed by RSM implementation for such cases).
   
   One way to solve is it is to delete the data that we know for sure is ready for deletion, e.g. if we have 10MB of data in remote store for non-active lineage and retention size is 2MB, then we can safely delete the rest of the 8MB. This is because even if this leadership chain becomes active, it will adhere to retention size.
   
   In other words, I am not saying that we should not delete non-active lineage data in remote store. I am saying that the non-active lineage data should only be deleted if it when it is violating the retention policies. If we have time based retention, this will ensure that there are no leaks. If we have size based retention, then we can do what you are suggesting.
   
   I will not consider this comment as blocking to merge this PR since this is in early access but we should document this risk of data loss as part of release notes and try to arrive at a conclusion before production release.
   
   Thoughts @showuon @junrao @ivanyu @jeqo ?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295701347


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,
+                                                            long logEndOffset,
+                                                            NavigableMap<Integer, Long> leaderEpochs) {

Review Comment:
   Thanks Jun for pointing it out.
   Currently, segment epochs are created from leader epoch cache truncated with start and end offsets. But I added defensive checks to filter the epoch with empty records as they will not have any records/messages in the segments. These changes with UTs added in the latest commits. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1680370332

   Thanks @junrao for the review. Addressed them with inline comments and/or with the latest commits. 
   
   >Also, do we have any test covering the newly introduced local retention logic?
   
   Will add more UTs. We will have integration tests in a followup PR once https://github.com/apache/kafka/pull/14116 is merged.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1299184981


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   This is only applicable with tiered storage enabled topics. We will add that in the release notes of tiered storage section about the change in the behavior.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1299221576


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   What's the behavior of unclean leader election when tiered storage is enabled?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293066497


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Good point! I think it's worth filing a bug in JIRA. WDYT @satishd ?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293086543


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   To ensure consistency, similar to local, which marks the segment for deletion (renames the file to .delete) and deletes it after 1 minute. (segment.delete.delay.ms). Should we move the log-start-offset before the remote log segment deletion?
   
   One way to do this is not to delete the remote log segments in `deleteRetentionTimeBreachedSegments` and `deleteRetentionTimeBreachedSegments` and only move the `logStartOffset`. In the next iteration, those remote-log-segments will be removed via `deleteLogStartOffsetBreachedSegments`.
   
   WDYT? @satishd



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294604024


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};

Review Comment:
   What is the rationale for this suggestion?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300199098


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   Sure. We can address this separately but I think that should be a blocker Jira for 3.6. Otherwise we are shipping this PR with a known bug which I am not very comfortable with. This bug is also not very edge case-y as others for which we have started Jira items such as bugs related to performance instead this bug impacts correctness. 
   
   Do you agree?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300279331


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   Yes, this is planned for 3.6.0. I did not want to block this PR with that as we want to unblock other dependent PRs, especially integration test PRs.  



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1573234538

   Thanks @junrao for your review comments. Addressed them inline and/or with the latest commits. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1576564925

   Pulled the latest trunk, resolved the conflicts, and pushed the changes.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1670963444

   Thanks @divijvaidya for the review. Addressed your comments inline or with the latest commits. 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1247799637


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -152,19 +156,23 @@ public class RemoteLogManager implements Closeable {
      * @param time      Time instance.
      * @param clusterId The cluster id.
      * @param fetchLog  function to get UnifiedLog instance for a given topic.
+     * @param updateRemoteLogStartOffset function to update the log-start-offset for a given topic partition.
      */
     public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
                             int brokerId,
                             String logDir,
                             String clusterId,
                             Time time,
-                            Function<TopicPartition, Optional<UnifiedLog>> fetchLog) {
+                            Function<TopicPartition, Optional<UnifiedLog>> fetchLog,
+                            BiConsumer<TopicPartition, Long> updateRemoteLogStartOffset) {

Review Comment:
   nit: 
   ```suggestion
                               BiConsumer<TopicPartition, Long> updateLogStartOffsetFromRemoteTier) {
   ```



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -634,6 +642,241 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;

Review Comment:
   maybe worth adding a log message here stating why cleanup is not happening? or maybe just a comment explaining why this scenario may never happen given the low prob that recordVersion < 2 is used. 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   @satishd , could you elaborate a bit more what do you mean by
   > Respective garbage collectors in those storages will take care of deleting the data asynchronously.
   
   ?
   Is this relying on some specific storage backend implementation? 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   > even if we are not the leader at this stage, we have deleted the logs in remote. 
   
   If I'm reading the call path correctly, this is not the case. `handleLogStartOffsetUpdate` function is called only at the end of `cleanupExpiredRemoteLogSegments` that filters out calls from followers.
   I guess we could either remote the `isLeader` validation here, or move this logic within the lambda itself?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -634,6 +642,241 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {

Review Comment:
   Could we add a log info here similar to copy?
   ```suggestion
                   if (predicate.test(segmentMetadata)) {
                       logger.info("Deleting remote log segment {}", metadata.remoteSegmentId());
   ```



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -922,6 +947,10 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  private def maybeIncrementLocalLogStartOffset(newLogStartOffset: Long, reason: LogStartOffsetIncrementReason): Unit = {
+    maybeIncrementLogStartOffset(newLogStartOffset, reason, onlyLocalLogStartOffsetUpdate = true)
+  }

Review Comment:
   found this a bit confusing. In main operation `onlyLocalLogStartOffsetUpdate` is false by default, but here we are overriding with `onlyLocalLogStartOffsetUpdate` as true, and methods signature are mainly the same. Wouldn't be clearer to use the default method with `onlyLocalLogStartOffsetUpdate=true` instead of creating this private method?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1284241897


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {

Review Comment:
   This is not required as the updated code does not use this method. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295402786


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};

Review Comment:
   For clean code, it creates an anonymous extra class at every usage and we should try to avoid this pattern.
   
   https://www.baeldung.com/java-initialize-hashmap



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300055517


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   This takes an assumption that the partition has continuous monotonically increasing offsets. But it is not true for a topic that was historically compacted (i.e. compaction is turned off now, that is why TS is enabled).
   
   I would suggest to read the next segment and set the startOffset as the start offset of the next segment.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300176481


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   Good point. I think we discussed this earlier scenario. Let us address this in a followup PR covering topics changing their retention from compaction to delete only retention. Filed https://issues.apache.org/jira/browse/KAFKA-15388



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1555964010

   Actually, the proposed approach in this PR avoids inconsistency issues as mentioned [here](https://github.com/apache/kafka/pull/13561#discussion_r1181526976) and also avoids remote segment leaks in unclean leader election scenarios.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1234978872


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1331,7 +1370,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage

Review Comment:
   It is inclusive, updated with the doc describing about the variable.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213100593


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {

Review Comment:
   `listRemoteLogSegments(TopicIdPartition topicIdPartition)` are not returned in any specific order. 
   



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {

Review Comment:
   `listRemoteLogSegments(TopicIdPartition topicIdPartition)` are not returned in any specific order. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1291690729


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Here is what I mean. Ideally, the retention behavior should be unchanged with remote storage. Consider the following case without remote storage. Let's say retentionSize is 100MB and we have only 1 segment of 90MB. The retention logic won't trigger the deletion of the last segment.
   
   Now, consider the same situation with remote storage enabled, but no remote segments. If localRetention is 20MB, the retention logic will delete last segment of 90MB. Since the data is not in remote storage. We have deleted the data a bit earlier than expected.
   
   A similar issue exists for time-based retention. If remote storage is enabled, but no remote segments, the time-based retention is now based on localRentionTime, not retentionTime. Since the former can be smaller than the latter, it means that we could delete the data earlier than expected.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   Thanks for the explanation. It makes sense to me. Could you add a comment that this is needed for unclean leader election?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage
+        else true

Review Comment:
   In that case, the name `isSegmentTieredToRemoteStorage` is a misnomer. If remote storage is disabled, there shouldn't be any segment tiered to remote storage, yet we are setting this val to true.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -162,20 +167,23 @@ public class RemoteLogManager implements Closeable {
      * @param time      Time instance.
      * @param clusterId The cluster id.
      * @param fetchLog  function to get UnifiedLog instance for a given topic.
+     * @param updateRemoteLogStartOffset function to update the log-start-offset for a given topic partition.

Review Comment:
   While you are here, could you also add the missing javadoc for brokerTopicStats?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1464,12 +1513,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled()))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
+      nextSegmentOpt.exists(_.baseOffset <= (if(remoteLogEnabled()) localLogStartOffset() else logStartOffset))

Review Comment:
   space after `if`.
   
   Also, this logic still doesn't look quite right. If remote log is enabled, it seems that we still want to delete local segments whose offset is smaller than logStartOffset.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289577043


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   This covers scenarios where unclean leader election happens and the remote storage contains segments that are earlier to the current leader's leader-epoch-lineage. 
   
   For ex:
   
   The current leader has the current leader-epoch-cache.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        3              700
        4              780
        6              900
        7              990 					
   -----------------------------
   ```
   But the earlier broker which got replaced with a new broker which has the current leader's leader-epoch lineage.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        0              0
        1              180
        2              400
   -----------------------------
   ```
   But these segments did not expire retention and they were not deleted in the remote storage. But these leader epochs are not there in the current leader's leader epoch as it was chosen with unclean leader election. In this case, we need to remove the segments, that do not exist beyond the current leader epoch lineage. Otherwise, they will never be cleaned up and will continue to accumulate in remote storage.
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289577043


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   This covers scenarios where unclean leader election happens and the remote storage contains segments that are earlier to the current leader's leader-epoch-lineage. 
   
   For ex:
   
   The current leader has the current leader-epoch-cache.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        3              700
        4              780
        6              900
        7              990 					
   -----------------------------
   ```
   But the earlier broker which got replaced with a new broker which has the current leader's leader-epoch lineage.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        0              0
        1              180
        2              400
   -----------------------------
   ```
   But these segments did not expire retention and they were not deleted in the remote storage. But these leader epochs are not there in the current leader's leader epoch as it was chosen with unclean leader election. In this case, we need to remove the segments, that exist beyond the current leader epoch lineage. Otherwise, they will never be cleaned up and will continue to accumulate in remote storage.
   
   



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   This covers scenarios where unclean leader election happens and the remote storage contains segments that are earlier to the current leader's leader-epoch-lineage. 
   
   For ex:
   
   The current leader has the current leader-epoch-cache.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        3              700
        4              780
        6              900
        7              990 					
   -----------------------------
   ```
   But the earlier broker which got replaced with a new broker which has the current leader's leader-epoch lineage.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        0              0
        1              180
        2              400
   -----------------------------
   ```
   But these segments did not expire retention and they were not deleted in the remote storage. But these leader epochs are not there in the current leader's leader epoch as it was chosen with unclean leader election. In this case, we need to remove the segments, that exist beyond the current leader epoch lineage. Otherwise, they will never be cleaned up and will continue to accumulate in remote storage.
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293071660


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   So you mean, all the segment deletion will happen again in the new leader, and update the log start offset there. OK, make sense.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293081560


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   In the FETCH response, the leader-log-start-offset will be piggy-backed. But, there can be a scenario:
   
   1. Leader deleted the remote log segment and updates it's log-start-offset
   2. Before the replica-2 update it's log-start-offset via FETCH-request, the leadership changed to replica-2.
   3. There are no more eligible segments to delete from remote.
   4. The log-start-offset will be stale (referring to old log-start-offset but the data was already removed from remote)
   5. If the consumer starts to read from the beginning of the topic, it will fail to read.
   
   It's good to handle this case in this PR itself.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14888: Added remote log segments retention functionality based on time and size. [kafka]

Posted by "iit2009060 (via GitHub)" <gi...@apache.org>.
iit2009060 commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1433944692


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   @divijvaidya  @satishd  @showuon  I gone through the specific code and realised this is actually not impacting the logic
   1. While copying the remote segments , remotelogsegmentmetadata stores  endoffset using value from the nextSegment base offset.
   https://github.com/apache/kafka/blob/5785796f985aa294c12e670da221d086a7fa887c/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L693
   2.  In my understanding it will be safe to use same logic for historically compacted topics. 
   Let me know  If my  analysis is correct or not ?
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] KAFKA-14888: Added remote log segments retention functionality based on time and size. [kafka]

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1434040959


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   Yes that is correct. Copying functionality is not impacted as discussed in https://issues.apache.org/jira/browse/KAFKA-15388. It's only the read-from-remote that is impacted for the historically compacted topic.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1665486770

   Thanks @showuon for the review. Addressed them with the latest commits.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213112946


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -154,16 +154,41 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()

Review Comment:
   We already set the localLogStartOffset as max of passed logStartOffset and the first segment's base offset. 
   When remote log is not enabled, `logStartOffset` is set as `localLogStartOffset` as computed above.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213113458


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1383,20 +1421,30 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  private[log] def localRetentionMs: Long = {
+    if (config.remoteLogConfig.remoteStorageEnable) config.remoteLogConfig.localRetentionMs else config.retentionMs
+  }
+
   private def deleteRetentionMsBreachedSegments(): Int = {
-    if (config.retentionMs < 0) return 0
+    val retentionMs = localRetentionMs

Review Comment:
   This is addressed with the latest commits. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289566534


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Local log size is based on the local retention configs and those are always less than or equal to the complete log retention.
   
   I'm unclear about the rationale behind retaining data in local storage using an overall retention size where there are no remote log segments. Please provide clarification.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295607067


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Thanks @junrao for the clarification. 
   
   In the above case with remote storage enabled, it will eventually be deleted from local and remote storages, and updates log-start-offset and local-log-start-offset respectively. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1292935520


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage
+        else true

Review Comment:
   > In that case, the name isSegmentTieredToRemoteStorage is a misnomer.
   
   Nice catch!



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293345266


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Let me rephrase what you mentioned here
   
   retention.bytes= 100MB
   segment1 - 90MB
   
   When remote storage is not enabled, then this segment is not deleted from local log segments becuas eof the retention size check.
   
   retention.bytes= 100MB
   local.retention.bytes= 20MB
   segment1 - 90MB
   
   When remote storage is enabled, and there are no segments uploaded to remote storage. That means it will not allow this segment to be deleted as it is not yet copied to remote storage based on the introduced check in this PR.
   
   If it is copied to remote storage, that means it is not an active segment and there are one or more local segments after this segment. This segment will be eligible for deletion based on the local retention policy as it is already copied to remote storage earlier.
   
   @junrao Am I missing anything here?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] nikramakrishnan commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "nikramakrishnan (via GitHub)" <gi...@apache.org>.
nikramakrishnan commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293402836


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,

Review Comment:
   +1. We should add this check to [findOffsetByTimestamp](https://github.com/apache/kafka/blob/43751d8d0521b1440a823a9430fdb0659ce7c436/core/src/main/java/kafka/log/remote/RemoteLogManager.java#L434C12-L434C12) to ensure we select the segment with correct leader lineage.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293179175


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   @kamalcph The case mentioned by you can be addressed in a followup PR. Please file a JIRA.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293158752


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};
+
+        // Test whether a remote segment's epochs/offsets(multiple) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(
+                15,
+                35,
+                new TreeMap<Integer, Long>() {{
+                    put(1, 15L);
+                    put(2, 20L);
+                    put(3, 30L);
+                }}), logEndOffset, leaderEpochToStartOffset));
+
+        // Test whether a remote segment's epochs/offsets(single) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(
+                15,
+                19,
+                new TreeMap<Integer, Long>() {{
+                    put(1, 15L);
+                }}), logEndOffset, leaderEpochToStartOffset));
+
+        // Test whether a remote segment's epochs/offsets(single) are within the range of leader epochs
+        assertTrue(RemoteLogManager.isRemoteSegmentWithinLeaderEpochs(createRemoteLogSegmentMetadata(

Review Comment:
   Statements in L1065 and L1057 are same. Typo error?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {

Review Comment:
   `Optional` is not recommended as parameter in Java:
   
   https://stackoverflow.com/a/31923105



##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -1003,6 +1015,134 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                Optional.empty(),
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {
+        // Test whether a remote segment is within the leader epochs
+        final long logEndOffset = 90L;
+
+        TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<Integer, Long>() {{
+                put(0, 0L);
+                put(1, 10L);
+                put(2, 20L);
+                put(3, 30L);
+                put(4, 40L);
+                put(5, 50L);
+                put(7, 70L);
+            }};

Review Comment:
   nit:
   
   ```suggestion
           TreeMap<Integer, Long> leaderEpochToStartOffset = new TreeMap<>();
           leaderEpochToStartOffset.put(0, 0L);
           leaderEpochToStartOffset.put(1, 10L);
           leaderEpochToStartOffset.put(2, 20L);
           leaderEpochToStartOffset.put(3, 30L);
           leaderEpochToStartOffset.put(4, 40L);
           leaderEpochToStartOffset.put(5, 50L);
           leaderEpochToStartOffset.put(7, 70L);
   ```



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300178242


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1096,6 +1493,43 @@ public void close() {
         }
     }
 
+    // Visible for testing
+    public static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < 0)
+                throw new IllegalArgumentException("retentionSize should be non negative, but it is " + retentionSize);
+
+            if (remainingBreachedSize <= 0) {
+                throw new IllegalArgumentException("remainingBreachedSize should be more than zero, but it is " + remainingBreachedSize);
+            }
+
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+    }
+
+    // Visible for testing
+    public static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            if (retentionMs < 0)
+                throw new IllegalArgumentException("retentionMs should be non negative, but it is " + retentionMs);
+
+            if (retentionMs < cleanupUntilMs) {

Review Comment:
   This check will be true when using system time. But added this defensive check if we have tests setting the mock time to set any long values.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeqo commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "jeqo (via GitHub)" <gi...@apache.org>.
jeqo commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300314011


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1096,6 +1493,43 @@ public void close() {
         }
     }
 
+    // Visible for testing
+    public static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            if (retentionSize < 0)
+                throw new IllegalArgumentException("retentionSize should be non negative, but it is " + retentionSize);
+
+            if (remainingBreachedSize <= 0) {
+                throw new IllegalArgumentException("remainingBreachedSize should be more than zero, but it is " + remainingBreachedSize);
+            }
+
+            this.retentionSize = retentionSize;
+            this.remainingBreachedSize = remainingBreachedSize;
+        }
+    }
+
+    // Visible for testing
+    public static class RetentionTimeData {
+
+        private final long retentionMs;
+        private final long cleanupUntilMs;
+
+        public RetentionTimeData(long retentionMs, long cleanupUntilMs) {
+            if (retentionMs < 0)
+                throw new IllegalArgumentException("retentionMs should be non negative, but it is " + retentionMs);
+
+            if (retentionMs < cleanupUntilMs) {

Review Comment:
   > This check will be true when using system time.
   
   Yeah, but the part I'm missing is why should we throw an exception when this is true. If retention is 1 hour, and `cleanupUntil` is at any point in system time, we are throwing an exception.  



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1571615955

   @satishd 
   
   Will you open a separate PR to delete the active segment once it breaches the retention time? Or, will handle it in this patch.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1571973707

   > @satishd
   > 
   > Will you open a separate PR to delete the active segment once it breaches the retention time? Or, will handle it in this patch.
   
   I do not want to add more changes to the PR. Planned to have it in a followup PR, filed https://issues.apache.org/jira/browse/KAFKA-15047 


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dopuskh3 commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "dopuskh3 (via GitHub)" <gi...@apache.org>.
dopuskh3 commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1307788374


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            // Build the leader epoch map by filtering the epochs that do not have any records.
+            NavigableMap<Integer, Long> epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets());
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            long logStartOffset = log.logStartOffset();
+            long logEndOffset = log.logEndOffset();
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), logEndOffset, epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, logStartOffset);
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());

Review Comment:
   @satishd it seems I'm reaching that codepath when running reassignments on my cluster and segment are deleted from remote store despite a huge retention (topic created a few hours ago with 1000h retention). 
   It seems to happen consistently on some partitions when reassigning but not all partitions.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1203191600


##########
core/src/main/scala/kafka/server/BrokerServer.scala:
##########
@@ -522,7 +522,12 @@ class BrokerServer(
       }
 
       Some(new RemoteLogManager(remoteLogManagerConfig, config.brokerId, config.logDirs.head, time,
-        (tp: TopicPartition) => logManager.getLog(tp).asJava));
+        (tp: TopicPartition) => logManager.getLog(tp).asJava,
+        (tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
+          logManager.getLog(tp).foreach(log => {
+            log.updateLogStartOffsetFromRemoteTier(remoteLogStartOffset)
+          })
+        }));

Review Comment:
   unnecessary semicolon. Ditto in KafkaServer.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {

Review Comment:
   Should we exit the loop the first time a remote segment offset passes localLogStartOffset?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2128,7 +2182,7 @@ object LogMetricNames {
 
 case class RetentionMsBreach(log: UnifiedLog) extends SegmentDeletionReason {
   override def logReason(toDelete: List[LogSegment]): Unit = {
-    val retentionMs = log.config.retentionMs
+    val retentionMs = log.config.remoteLogConfig.localRetentionMs

Review Comment:
   retention time depends on remote storage being enabled, right? Ditto in line 2284.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -152,13 +156,15 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
                             int brokerId,
                             String logDir,
                             Time time,
-                            Function<TopicPartition, Optional<UnifiedLog>> fetchLog) {
+                            Function<TopicPartition, Optional<UnifiedLog>> fetchLog,
+                            BiConsumer<TopicPartition, Long> updateRemoteLogStartOffset) {

Review Comment:
    missing javadoc for new param.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the

Review Comment:
   Hmm, not sure that I follow the comment.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);

Review Comment:
   Are we iterating the same remote segment multiple times since a segment could have multiple epochs?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -154,16 +154,41 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {
+    _localLogStartOffset = offset
+
+    if (highWatermark < offset) {
+      updateHighWatermark(offset)
+    }
+
+    if (this.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)
+    }
+  }
+
+  def updateLogStartOffsetFromRemoteTier(remoteLogStartOffset: Long): Unit = {
+    if (!remoteLogEnabled()) {
+      warn("Ignoring the call as the remote log storage is disabled")

Review Comment:
    Should we return here to actually ignore?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;

Review Comment:
   What about overlapping segments between remote and local? Do we double count them?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -154,16 +154,41 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()

Review Comment:
   Hmm, logStartOffset could be larger than base offset of the first local segment. So, it seems that we can't just switch to the base offset of the first local segment if remote log is not enabled.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +

Review Comment:
    Loggering uses $ notation, which is for scala.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -942,13 +974,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          _localLogStartOffset = math.max(newLogStartOffset, localLogStartOffset())
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {
+            updatedLogStartOffset = true
+            updateLogStartOffset(newLogStartOffset)
+            _localLogStartOffset = newLogStartOffset

Review Comment:
   Yes, agreed. Not sure why we need to update _localLogStartOffset here again.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1383,20 +1421,30 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  private[log] def localRetentionMs: Long = {
+    if (config.remoteLogConfig.remoteStorageEnable) config.remoteLogConfig.localRetentionMs else config.retentionMs
+  }
+
   private def deleteRetentionMsBreachedSegments(): Int = {
-    if (config.retentionMs < 0) return 0
+    val retentionMs = localRetentionMs

Review Comment:
   Hmm, local retention needs to be bound by last tiered offset, right?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181520975


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   This is different from local log deletion. It requires the deletion of segments from local storage which need to really delete the files. 
   But incase of remote storages, it does not wait for the data to be deleted but it marks the file or object for deletion in their respective metadata stores. Respective garbage collectors in those storages will take care of deleting the data asynchronously. There is no perf impact for these delete calls as they take a much shorter time than copying segments. It is very unlikely that copying segments get affected because of the deletion of segments. Deletion checks are happening in every iteration so there will not be many segments that need to be deleted. 
   Anyways, we can discuss this separately in a separate JIRA. On another note, all this logic will go to UnifiedLog in future.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181526976


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   log.retention.<> configs indicate the total amount of log segments that can be stored in remote storage. So, it is not just about the segments only related to the current leader epoch lineage. We need to be careful of removing any unreferenced segments and also should not have any segment leaks in the remote storage incase of unclean leader elections. So, it cleans up any unreferenced segments beyond the earliest leader epoch that are also available for retention checks.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181520975


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   This is different from local log deletion. It requires the deletion of segments from local storage which need to really delete the files. But incase of remote storages, it does not wait for the data to be deleted but it marks the file or object for deletion in their respective metadata stores. Respective garbage collectors in those storages will take care of deleting the data asynchronously. There is no perf impact for these delete calls as they take much shorter time than copying segments. It is very unlikely that copying segments get affected because of deletion of segments. Deletion checks are happening in every iteration so there will not be many segments that need to be deleted. Anyways, we can discuss this separately in a separate JIRA. On another note, all this logic will go to UnifiedLog in future.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181532348


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata, checkTimestampRetention, cleanupTs, retentionMs) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata, checkSizeRetention) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            if (leaderEpochCache.earliestEntry().isPresent()) {

Review Comment:
   Unreferenced segments within the current leader epoch chain will eventually move earlier to the earliest epoch of the current leader epoch chain after a few retention checks. That will take care of those kinds of 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181520975


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   This is different from local log deletion. It requires the deletion of segments from local storage which need to really delete the files. B
   But incase of remote storages, it does not wait for the data to be deleted but it marks the file or object for deletion in their respective metadata stores. Respective garbage collectors in those storages will take care of deleting the data asynchronously. There is no perf impact for these delete calls as they take a much shorter time than copying segments. It is very unlikely that copying segments get affected because of the deletion of segments. Deletion checks are happening in every iteration so there will not be many segments that need to be deleted. 
   Anyways, we can discuss this separately in a separate JIRA. On another note, all this logic will go to UnifiedLog in future.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1197648143


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata, checkTimestampRetention, cleanupTs, retentionMs) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata, checkSizeRetention) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            if (leaderEpochCache.earliestEntry().isPresent()) {

Review Comment:
   I see, yes that's right those segments will eventually fall out of the range of active leader epochs. That should be fine, as long as users know there is no specific enforcement on the time those unreferenced segments will be cleaned up.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1205425110


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);

Review Comment:
   debug("Update {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset)



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",

Review Comment:
   additional `$` sign: remote log segment [$]{} ...



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1224420683


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        return;

Review Comment:
   please add an info log on why we exited the function prior to it's completion. It greatly helps debugging when we don't have to guess where the return point was.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {

Review Comment:
   nit
   
   unnecessary else



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -945,4 +1176,27 @@ public void close() {
         }
     }
 
+    private static class RetentionSizeData {
+        private final long retentionSize;
+        private final long remainingBreachedSize;
+
+        public RetentionSizeData(long retentionSize, long remainingBreachedSize) {
+            this.retentionSize = retentionSize;

Review Comment:
   please perform an argument validation here. If retentionSize < remainingBreachedSize, then IllegalArgumentException.
   
   Same for RetentionTimeData



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;

Review Comment:
   please add a debug log here (and  other places where we are exiting this function) so that we know while debugging where did we exit the function from.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        return;
+                    }
+
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(boolean checkSizeRetention,
+                                                                   long retentionSize,
+                                                                   UnifiedLog log,
+                                                                   long totalSizeEarlierToLocalLogStartOffset) {
+            if (checkSizeRetention) {
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+                long remainingBreachedSize = totalSize - retentionSize;

Review Comment:
   please add the following check. We don't want to construct an object for RetentionSizeData if not required.
   
   ```
   if (totalSize > retentionSize) {
       long breachedSize = totalSize - retentionSize
       return Optional.of(new RetentionSizeData(retentionSize, remainingBreachedSize));
   }
   ```



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2144,7 +2208,7 @@ case class RetentionSizeBreach(log: UnifiedLog) extends SegmentDeletionReason {
     var size = log.size
     toDelete.foreach { segment =>
       size -= segment.size
-      log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+      log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config)} breach. Log size " +

Review Comment:
   nit
   
   s/Log size after deletion/Local log size after deletion
   
   Asking so that the reader can disambiguate between Log size (which is tiered + local) and local log size.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;

Review Comment:
   we need to restore the original value of remainingBreachedSize when remainingBreachedSize < 0?
   
   May I suggest re-writing this entire predicate here as:
   ```
   long segmentSize = metadata.segmentSizeInBytes()
   remainingBreachedSize -= segmentSize
   if (remainingBreachedSize < 0) {
       remainingBreachedSize += segmentSize
       return false
   }
   
   return true
   ```
   
   Note that remainingBreachedSize is a member of the class and you don't need to do `retentionSizeData.get().remainingBreachedSize`. Also the earlier `if (retentionSizeData.get().remainingBreachedSize > 0) {` is made redundant by the code I suggested.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   We are assuming that the state of local log will remain same from this point to the time we use the information computed here (i.e. totalSizeEarlierToLocalLogStartOffset ) to delete the segments. But that is not true since local retention threads are running concurrently and might have moved the localLogStartOffset by the time we use the `totalSizeEarlierToLocalLogStartOffset` computed here. As an example:
   
   ### Time instant: T1
   LocalLSO = 10
   LSO = 0
   LSE = 20
   TieredEO = 15
   
   In this case we will calculate `totalSizeEarlierToLocalLogStartOffset` as the size from 0-10. 
   
   ### Time instant: T2
   Local log retention thread deletes some stuff and updates the LocalLSO=14
   
   ### Time instant: T3
   When we calculate `long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;` at `buildRetentionSizeData`,  
   validLocalLogSegmentsSize returns data from 14-20 and we say that the total size = totalSizeEarlierToLocalLogStartOffset ( i.e. 0-10) + validLocalLogSegmentsSize (i.e. 14-20).
   
   This leads to data from 11-13 not being counted anywhere. This looks like a bug! We need to re-use the values stores at the beginning of the retention calculation otherwise other threads (local retention threads) may change the values behind the scenes.
   
   Thoughts?
   



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   even if we are not the leader at this stage, we have deleted the logs in remote. Shouldn't we still update the metadata?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {
+    _localLogStartOffset = offset
+
+    if (highWatermark < offset) {
+      updateHighWatermark(offset)
+    }
+
+    if (this.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)
+    }
+  }
+
+  def updateLogStartOffsetFromRemoteTier(remoteLogStartOffset: Long): Unit = {
+    if (!remoteLogEnabled()) {
+      info("Ignoring the call as the remote log storage is disabled")

Review Comment:
   this should probably be a error level log because we don't expect to call this method when remote storage is disabled. Isn't that right?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (retentionSizeData.get().remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionSize = log.config().retentionSize;
+            final boolean checkSizeRetention = retentionSize > -1;
+
+            final long retentionMs = log.config().retentionMs;
+            final boolean checkTimestampRetention = retentionMs > -1;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (checkSizeRetention && segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(checkSizeRetention, retentionSize, log, totalSizeEarlierToLocalLogStartOffset);
+            Optional<RetentionTimeData> retentionTimeData = checkTimestampRetention
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);

Review Comment:
   Sorry, I am a bit confused here. Earlier in the comment https://github.com/apache/kafka/pull/13561#discussion_r1181526976 you mentioned that retention size/time configuration applies across all epochs. i.e. if I say retention is 3GB and the total log as per current epoch is 2 GB, but the total data stored in remote +local = 7GB, then I will delete (7-3) = 4GB of data as part of this cleanup. Is my understanding correct? If yes, then we seem to be deleting only the current leadership chain here BUT we are using the breached size from ALL the epochs calculated earlier. Isn't this contradictory?
   



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =

Review Comment:
   This comment has been addressed in the latest code



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {

Review Comment:
   should we ensure that we have acquired the partition `lock` first? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1596907836

   @satishd , any update for this PR? If you don't have time on it, just let me know. :)


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1573968832

   Hey @satishd 
   Are you planning to address the open comments such as https://github.com/apache/kafka/pull/13561/files#r1166767890 before I do another pass of code review?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1304597735


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   Syncedup with Jun to understand the comment here and clarified them. The retention logic deletes the segments with leader epochs preceding the earliest leader epoch in the current leader. Any epochs/offsets which are not there in the current leader epoch lineage but they are within the range, those will be eventually deleted when the current leader's earliest leader epoch moves beyond that.
   
   > In that case, when an unclean leader is elected, does it just use its logEndOffset to start writing new data?
   
   Right, it will start writing with its logEndoffset with the new epoch.
   
   > In that case, do we just hide those remote segments with offsets higher than the new leader's starting logEndOffset? Will those hidden remote segments be cleaned up eventually?
   
   Right, they will be eventually removed. 
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1281844159


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -945,13 +978,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {

Review Comment:
   Good point! Addressed in the latest commits to keep the logic simpler.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1298029817


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   That is a fair point. If we want to take that approach, we should not delete any segments beyond the current leader's leader epoch lineage. We need to take the risk of RSM plugins having cleanup mechanisms for the segment leaks in the remote storage. These leaks may accumulate over time and create operational issues. It is hard even for RSM plugin owners to deterministically find out whether a segment is unreferenced when there are out-of-sync/offline replicas.
   
   I filed [KAFKA-15376](https://issues.apache.org/jira/browse/KAFKA-15376) to continue the discussion and take a final call later before productionizing 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1294602887


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {

Review Comment:
   I do not find a strong reason not to use Optional as an argument. :) In the same SO link, few other opinions on why it is a weak argument. 
   
   Optional as an argument is used in several other places within this project. I do not have strong opinions and I am fine if we decide to go with that rule across the project when there is a consensus. We can revisit it when we do that.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293081560


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   In the FETCH response, the leader-log-start-offset will be piggy-backed. But, there can be a scenario:
   
   1. Leader deleted the remote log segment and updates it's log-start-offset
   2. Before the replica-2 update it's log-start-offset via FETCH-request, the leadership changed to replica-2.
   3. There are no more eligible segments to delete from remote.
   4. The log-start-offset will be stale (referring to old log-start-offset but the data was already removed from remote)
   5. If the consumer starts to read from the beginning of the topic, it will fail to read.
   
   We can take this task in a follow-up PR if required.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] kamalcph commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "kamalcph (via GitHub)" <gi...@apache.org>.
kamalcph commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293086543


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +625,230 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {

Review Comment:
   Similar to local, which marks the segment for deletion (renames the file to .delete) and deletes it after 1 minute. (segment.delete.delay.ms). Should we move the log-start-offset before the remote log segment deletion?



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293344902


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Let me rephrase what you mentioned here
   
   retention.bytes= 100MB
   segment1 - 90MB
   
   When remote storage is not enabled, then this segment is not deleted from local log segments becuas eof the retention size check. 
   
   retention.bytes= 100MB
   local.retention.bytes= 20MB
   segment1 - 90MB
   
   When remote storage is enabled, and there are no segments uploaded to remote storage. That means it will not allow this segment to be deleted as it is not yet copied to remote storage based on the introduced check in this PR. 
   
   If it is copied to remote storage, that means it is not an active segment and there are one or more local segments after this segment. This segment will be eligible for deletion based on the local retention policy as it is already copied to remote storage earlier.
   
   @junrao Am I missing antyhing here?
   



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   Let me rephrase what you mentioned here
   
   retention.bytes= 100MB
   segment1 - 90MB
   
   When remote storage is not enabled, then this segment is not deleted from local log segments becuas eof the retention size check. 
   
   retention.bytes= 100MB
   local.retention.bytes= 20MB
   segment1 - 90MB
   
   When remote storage is enabled, and there are no segments uploaded to remote storage. That means it will not allow this segment to be deleted as it is not yet copied to remote storage based on the introduced check in this PR. 
   
   If it is copied to remote storage, that means it is not an active segment and there are one or more local segments after this segment. This segment will be eligible for deletion based on the local retention policy as it is already copied to remote storage earlier.
   
   @junrao Am I missing antyhing here?
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1296732476


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   In case of unclean leader election, there is already a durability loss when a non in-sync replica needs to be chosen as a leader and given preference to availability. The approach taken in this PR uses the current tradeoff of durability loss and avoids remote log segment leaks. This is slightly different from local log cleanup which we can clarify in the release notes. 
   
   Retention/cleanup logic spread across multiple layers(outside of Kafka) poses significant risks and could lead to more extensive problems. So, it is better that to be handled by Kafka's retention mechanism.
   
   We will discuss further on finalizing the approach before we make this feature production ready. 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300176481


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);

Review Comment:
   Good point. I think we discussed this earlier also. Let us address this in a followup PR covering topics changing their retention from compaction to delete only retention. Filed https://issues.apache.org/jira/browse/KAFKA-15388



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1300372620


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   Hmm, I was just asking about how unclean leader election with tiered storage is handled in 3.6.0. It seems that this PR has removed the logic for retention by leader epoch. In that case, when an unclean leader is elected, does it just use its logEndOffset to start writing new data? In that case, do we just hide those remote segments with offsets higher than the new leader's starting logEndOffset? Will those hidden remote segments be cleaned up eventually? 



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] Hangleton commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "Hangleton (via GitHub)" <gi...@apache.org>.
Hangleton commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1167074461


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =
+                            remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata, checkTimestampRetention, cleanupTs, retentionMs) ||
+                                    remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata, checkSizeRetention) ||
+                                    remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            if (leaderEpochCache.earliestEntry().isPresent()) {

Review Comment:
   Hmm, I don't think it is enough to evict the data for epochs < than the current leader's smallest epoch. With unclean leader election, it is possible to have divergence in-between a log prefix and suffix shared by two replicas.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);

Review Comment:
   Hmm, it seems we are iterating over all remote segment metadata every time the expiration task is executed. This could become costly if the RLMM implementation does not cache the said metadata. That could be an explicit implementation constraint for plugin providers. Maybe we could also add a small layer a memoization here to avoid traversing the log metadata every time.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1166622209


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   Should this operation be performed in a separate thread pool which can have a defined quota? (similar to how we perform cleaning for local log using separate cleaner/background threads).
   
   I am concerned that this may impact the rate of copy to remote if amount of cleaning is large. Also, it's perhaps better to have different scaling characteristics for cleaning from remote vs. copying. Copying maybe considered urgent since slowness in copying can potentially fill up disk whereas cleaning from remote may be a lower priority activity.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {

Review Comment:
   We don't need to calculate this for time based retention. Right? If yes, can we refactor the code here so that we perform size calculation (since it requires a full scan over all log segments) only when it's required i.e. for size based retention.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);

Review Comment:
   we are only interested in segments in state COPY_SEGMENT_FINISHED or DELETE_SEGMENT_STARTED here. Right? (DELETE_SEGMENT_STARTED to clean up any stragglers)
   
   Can we make this more explicit by filtering on them?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);

Review Comment:
   please guard this update with `isLeader()`



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+                    isSegmentDeleted =

Review Comment:
   could we check for `if (isCancelled() || !isLeader())` here again please to short circuit this expensive loop during shutdown.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each

Review Comment:
   I would appreciate your response on https://lists.apache.org/thread/pc6x3xr4b8y84g43dcbdzh170kb5oz1v where we discussed that even with caching mechanism, warm up of the cache is going to slow down the copying. I agree that this can be discussed outside the scope of this PR but adding the above thread as FYI.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1378,20 +1416,30 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  private[log] def localRetentionMs: Long = {

Review Comment:
   I would suggest to have address local retention in a separate PR. We can limit this PR to handling remote log retention only.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -154,16 +154,41 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {
+    _localLogStartOffset = offset
+
+    if (highWatermark < offset) {
+      updateHighWatermark(offset)
+    }
+
+    if (this.recoveryPoint < offset) {
+      localLog.updateRecoveryPoint(offset)
+    }
+  }
+
+  def updateLogStartOffsetFromRemoteTier(remoteLogStartOffset: Long): Unit = {
+    if (!remoteLogEnabled()) {
+      warn("Ignoring the call as the remote log storage is disabled")

Review Comment:
   Do we need this at a warn level?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.

Review Comment:
   Unreferenced segments are not only the ones which have a lower epoch that earliest known epoch, they could also be ones which have an epoch that is not part of active epoch chain. How are we handling that?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())

Review Comment:
   `isCancelled() || !isLeader()`
   
   Also, please add a log which can help operator understand that this was actually cancelled. 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {

Review Comment:
   The current logic may cause deletion of more data than anticipated.
   
   This is because it is possible to have remote segments satisfying this condition which are not part of the current leadership epoch chain. Calculation of totalSizeEarlierToLocalLogStartOffset may include these segments as well and hence, the calculation of totalSize will include local segments + all remote segments (< local log start offset). 
   The calculated TotalSize will be actually larger than the actual total size (where actual total size = size of remote + local log for the active epoch chain). This will lead to higher value of remainingBreachedSize than actual and hence, more data gets deleted than necessary.
   
   Is this making sense? Else I can provide an example to explain it better.
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1181527119


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -595,6 +609,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);

Review Comment:
   We do not need any specific check here as we want to clean up any segment that is not yet deleted including COPY_SEGMENT_STARTED



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] divijvaidya commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "divijvaidya (via GitHub)" <gi...@apache.org>.
divijvaidya commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1180254846


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -581,11 +588,18 @@ public void run() {
                 if (isLeader()) {
                     // Copy log segments to remote storage
                     copyLogSegmentsToRemote();
+                    // Cleanup/delete expired remote log segments
+                    cleanupExpiredRemoteLogSegments();

Review Comment:
   That is fair. As I mentioned in the [review overview](https://github.com/apache/kafka/pull/13561#pullrequestreview-1385125558), I am fine (and would actually prefer) with creating JIRAs and tackling the perf related comments outside this PR. With this comment, I wanted to make sure we are aware and are tracking things that need fixing in this code.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1295713103


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   Follower replicas do truncation based on leader epoch lineage and catch up with the leader. It is hard to know whether a particular lineage can exist in any of the replicas as replicas can fail and it is hard to say whether a particular replica can come back with in a specific duration. That may cause leakages in remote storage. Follower replicas can not delete the remote segments as these may be part of the current leader and it may delete the data that is expected by the leader.  
   
   The tradeoff taken in case of unclean leader election here is to clean up the epoch lineage earlier to the current leader epoch instead of creating segment leakages in remote storage.
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1597054329

   Rebased with the trunk as it had conflicts with the trunk because of other introduced changes in trunk.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1665489584

   @junrao Thanks for the review. Addressed your comments inline or with the latest commits.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1281657244


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -667,11 +675,323 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,
+                                                            long logEndOffset,
+                                                            NavigableMap<Integer, Long> leaderEpochs) {
+        long segmentEndOffset = segmentMetadata.endOffset();
+        NavigableMap<Integer, Long> segmentLeaderEpochs = segmentMetadata.segmentLeaderEpochs();
+        // Check for out of bound epochs between segment epochs and current leader epochs.
+        Integer segmentFirstEpoch = segmentLeaderEpochs.firstKey();
+        Integer segmentLastEpoch = segmentLeaderEpochs.lastKey();
+        if (segmentFirstEpoch < leaderEpochs.firstKey() || segmentLastEpoch > leaderEpochs.lastKey()) {
+            LOGGER.debug("Remote segment {} is not within the partition leader epoch lineage. Remote segment epochs: {} and partition leader epochs: {}",

Review Comment:
   Should we log partition info as below did here?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -667,11 +675,323 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");

Review Comment:
   As commented above, there might be chances that the leadership change during the segment deletion, I think we should update the log start offset before exiting the `cleanupExpiredRemoteLogSegments` method since if there's no deletion happened, the `logStartOffset` will be empty. WDYT?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -979,13 +1010,19 @@ class UnifiedLog(@volatile var logStartOffset: Long,
 
         localLog.checkIfMemoryMappedBufferClosed()
         if (newLogStartOffset > logStartOffset) {
-          updatedLogStartOffset = true
-          updateLogStartOffset(newLogStartOffset)
-          _localLogStartOffset = newLogStartOffset
-          info(s"Incremented log start offset to $newLogStartOffset due to $reason")
-          leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
-          producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
-          maybeIncrementFirstUnstableOffset()
+          _localLogStartOffset = math.max(newLogStartOffset, localLogStartOffset())
+
+          // it should always get updated  if tiered-storage is not enabled.
+          if (!onlyLocalLogStartOffsetUpdate || !remoteLogEnabled()) {
+            updatedLogStartOffset = true
+            updateLogStartOffset(newLogStartOffset)
+            info(s"Incremented log start offset to $newLogStartOffset due to $reason")
+            leaderEpochCache.foreach(_.truncateFromStart(logStartOffset))
+            producerStateManager.onLogStartOffsetIncremented(newLogStartOffset)
+            maybeIncrementFirstUnstableOffset()
+          } else {
+            info(s"Incrementing local log start offset to ${localLogStartOffset()}")

Review Comment:
   Why don't we log `reason` here?



##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -823,6 +826,108 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() {
         }
     }
 
+    private static RemoteLogSegmentMetadata createRemoteLogSegmentMetadata(long startOffset, long endOffset, Map<Integer, Long> segmentEpochs) {
+        return new RemoteLogSegmentMetadata(
+                new RemoteLogSegmentId(new TopicIdPartition(Uuid.randomUuid(),
+                        new TopicPartition("topic", 0)), Uuid.randomUuid()),
+                startOffset, endOffset,
+                100000L,
+                1,
+                100000L,
+                1000,
+                RemoteLogSegmentState.COPY_SEGMENT_FINISHED, segmentEpochs);
+    }
+
+    @Test
+    public void testRemoteSegmentWithinLeaderEpochs() {

Review Comment:
   Nice test!



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2207,7 +2267,7 @@ case class RetentionSizeBreach(log: UnifiedLog) extends SegmentDeletionReason {
     var size = log.size
     toDelete.foreach { segment =>
       size -= segment.size
-      log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+      log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config)} breach. Local log size " +

Review Comment:
   nit: It's weird to see `local log retention size` when user is not enabled the tiered storage. Could we add a if check to see if remote storage is enabled or not and print the log accordingly?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -667,11 +675,323 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are lesser than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,
+                                                            long logEndOffset,
+                                                            NavigableMap<Integer, Long> leaderEpochs) {
+        long segmentEndOffset = segmentMetadata.endOffset();
+        NavigableMap<Integer, Long> segmentLeaderEpochs = segmentMetadata.segmentLeaderEpochs();
+        // Check for out of bound epochs between segment epochs and current leader epochs.
+        Integer segmentFirstEpoch = segmentLeaderEpochs.firstKey();
+        Integer segmentLastEpoch = segmentLeaderEpochs.lastKey();
+        if (segmentFirstEpoch < leaderEpochs.firstKey() || segmentLastEpoch > leaderEpochs.lastKey()) {
+            LOGGER.debug("Remote segment {} is not within the partition leader epoch lineage. Remote segment epochs: {} and partition leader epochs: {}",
+                    segmentMetadata.remoteLogSegmentId(), segmentLeaderEpochs, leaderEpochs);
+            return false;
+        }
+
+        for (Map.Entry<Integer, Long> entry : segmentLeaderEpochs.entrySet()) {
+            int epoch = entry.getKey();
+            long offset = entry.getValue();
+
+            // If segment's epoch does not exist in the leader epoch lineage then it is not a valid segment.
+            if (!leaderEpochs.containsKey(epoch)) {
+                LOGGER.debug("[{}]  Remote segment {}'s epoch {} is not within the leader epoch lineage. Remote segment epochs: {} and partition leader epochs: {}",
+                        segmentMetadata.topicIdPartition(), segmentMetadata.remoteLogSegmentId(), epoch, segmentLeaderEpochs, leaderEpochs);
+                return false;
+            }
+
+            // Segment's first epoch's offset can be more than or equal to the respective leader epoch's offset.
+            if (epoch == segmentFirstEpoch && offset < leaderEpochs.get(epoch)) {
+                LOGGER.debug("[{}]  Remote segment {}'s first epoch {}'s offset is more than leader epoch's offset {}.",
+                        segmentMetadata.topicIdPartition(), segmentMetadata.remoteLogSegmentId(), epoch, leaderEpochs.get(epoch));

Review Comment:
   The comment is not clear:
   `// Segment's first epoch's offset [should] be more than or equal to the respective leader epoch's offset.`
   
   The log is not correct:
   `"[{}]  Remote segment {}'s first epoch {}'s offset is [less] than leader epoch's offset {}.",`



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -152,16 +152,42 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   locally {
     initializePartitionMetadata()
     updateLogStartOffset(logStartOffset)
+    updateLocalLogStartOffset(math.max(logStartOffset, localLog.segments.firstSegmentBaseOffset.getOrElse(0L)))
+    if (!remoteLogEnabled())
+      logStartOffset = localLogStartOffset()
     maybeIncrementFirstUnstableOffset()
     initializeTopicId()
 
     logOffsetsListener.onHighWatermarkUpdated(highWatermarkMetadata.messageOffset)
+
+    info(s"Completed load of log with ${localLog.segments.numberOfSegments} segments, local log start offset ${localLogStartOffset()} and " +
+      s"log end offset $logEndOffset")
   }
 
   def setLogOffsetsListener(listener: LogOffsetsListener): Unit = {
     logOffsetsListener = listener
   }
 
+  private def updateLocalLogStartOffset(offset: Long): Unit = {

Review Comment:
   +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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1292967405


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =
+        if (remoteLogEnabled()) upperBoundOffset > 0 && upperBoundOffset - 1 <= highestOffsetInRemoteStorage
+        else true

Review Comment:
   It was implicit from the condition that it is relevant only when remote storage is enabled. I removed the value and added a condition and the respective comments for better clarity.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1292967294


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1464,12 +1513,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled()))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
+      nextSegmentOpt.exists(_.baseOffset <= (if(remoteLogEnabled()) localLogStartOffset() else logStartOffset))

Review Comment:
   When remote log is enabled, it deletes the local segments whose offset is <= local-log-start-offset. The existing condition without tiered storage is to delete the local log segments <= log-start-offset.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] junrao commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "junrao (via GitHub)" <gi...@apache.org>.
junrao commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1293823449


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -2202,19 +2262,24 @@ case class RetentionMsBreach(log: UnifiedLog) extends SegmentDeletionReason {
   }
 }
 
-case class RetentionSizeBreach(log: UnifiedLog) extends SegmentDeletionReason {
+case class RetentionSizeBreach(log: UnifiedLog, remoteLogEnabled: Boolean) extends SegmentDeletionReason {
   override def logReason(toDelete: List[LogSegment]): Unit = {
     var size = log.size
     toDelete.foreach { segment =>
       size -= segment.size
-      log.info(s"Deleting segment $segment due to retention size ${log.config.retentionSize} breach. Log size " +
+      if (remoteLogEnabled) log.info(s"Deleting segment $segment due to local log retention size ${UnifiedLog.localRetentionSize(log.config)} breach. " +

Review Comment:
   Should we do the same for RetentionMsBreach to log whether the retention time is for local retention or not?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1464,12 +1513,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled()))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
+      nextSegmentOpt.exists(_.baseOffset <= (if(remoteLogEnabled()) localLogStartOffset() else logStartOffset))

Review Comment:
   Yes, it just means that the segment won't be deleted until it's uploaded to the remote store. But this is probably ok.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -698,11 +707,329 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());
+                    }
+                }
+            }
+
+            // Update log start offset with the computed value after retention cleanup is done
+            remoteLogRetentionHandler.logStartOffset.ifPresent(offset -> handleLogStartOffsetUpdate(topicIdPartition.topicPartition(), offset));
+        }
+
+        private Optional<RetentionTimeData> buildRetentionTimeData(long retentionMs) {
+            return retentionMs > -1
+                    ? Optional.of(new RetentionTimeData(retentionMs, time.milliseconds() - retentionMs))
+                    : Optional.empty();
+        }
+
+        private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
+                                                                   long onlyLocalLogSegmentsSize,
+                                                                   long logEndOffset,
+                                                                   NavigableMap<Integer, Long> epochEntries) throws RemoteStorageException {
+            if (retentionSize > -1) {
+                long remoteLogSizeBytes = 0L;
+                for (Integer epoch : epochEntries.navigableKeySet()) {
+                    // remoteLogSize(topicIdPartition, epochEntry.epoch) may not be completely accurate as the remote
+                    // log size may be computed for all the segments but not for segments with in the current
+                    // partition's leader epoch lineage. Better to revisit this API.
+                    // remoteLogSizeBytes += remoteLogMetadataManager.remoteLogSize(topicIdPartition, epochEntry.epoch);
+                    Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsIterator.hasNext()) {
+                        RemoteLogSegmentMetadata segmentMetadata = segmentsIterator.next();
+                        if (isRemoteSegmentWithinLeaderEpochs(segmentMetadata, logEndOffset, epochEntries)) {
+                            remoteLogSizeBytes += segmentMetadata.segmentSizeInBytes();
+                        }
+                    }
+                }
+
+                // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+                // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+                long totalSize = onlyLocalLogSegmentsSize + remoteLogSizeBytes;
+                if (totalSize > retentionSize) {
+                    long remainingBreachedSize = totalSize - retentionSize;
+                    RetentionSizeData retentionSizeData = new RetentionSizeData(retentionSize, remainingBreachedSize);
+                    return Optional.of(retentionSizeData);
+                }
+            }
+
+            return Optional.empty();
+        }
+
         public String toString() {
             return this.getClass().toString() + "[" + topicIdPartition + "]";
         }
     }
 
+    /**
+     * Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     * The constraints here are as follows:
+     * - The segment's first epoch's offset should be more than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's end offset should be less than or equal to the respective leader epoch's offset in the partition leader epoch lineage.
+     * - The segment's epoch lineage(epoch and offset) should be same as leader epoch lineage((epoch and offset)) except
+     * for the first and the last epochs in the segment.
+     *
+     * @param segmentMetadata The remote segment metadata to be validated.
+     * @param logEndOffset    The log end offset of the partition.
+     * @param leaderEpochs    The leader epoch lineage of the partition.
+     * @return true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
+     */
+    // Visible for testing
+    public static boolean isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata segmentMetadata,
+                                                            long logEndOffset,
+                                                            NavigableMap<Integer, Long> leaderEpochs) {

Review Comment:
   We want to be a bit careful of using this method. LeaderEpochCache is mostly derived from the data in the log. However, on new leader epoch, the new leader also appends the new epoch to LeaderEpochCache before any record is appended for the epoch. This could cause a slight mis-match between the epoch chain in the remote segment and LeaderEpochCache. For example, it's possible for a LeaderEpochCache to have
   
   10 100
   11  200 //no record appended for epoch 11
   12 200
   
   where a segment's epoch chain only has 
   10 100
   12 200
   
   We don't want to prevent the remote segment from be deleted through the retention logic because of this slight mismatch on leader epoch chain.



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1390,7 +1424,15 @@ class UnifiedLog(@volatile var logStartOffset: Long,
   private def deleteOldSegments(predicate: (LogSegment, Option[LogSegment]) => Boolean,
                                 reason: SegmentDeletionReason): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      highWatermark >= nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset) &&
+      val upperBoundOffset = nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
+
+      // Check not to delete segments which are not yet copied to tiered storage
+      val isSegmentTieredToRemoteStorage =

Review Comment:
   @satishd : Sorry, I didn't give the right example. This is the case.
   
   Without remote storage,
     retention.bytes= 100MB
     segment1 - 200MB
   We will delete segment1 (even if it's the active segment).
   
   With remote storage, 
     retention.bytes= 100MB
     local.retention.bytes= 20MB
     segment1 - 200MB
   If segment1 is the active segment, it won't be deleted until it rolls and is uploaded to the remote store.
   
   It's a very subtle difference.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on PR #13561:
URL: https://github.com/apache/kafka/pull/13561#issuecomment-1692562057

   There are a few failures unrelated to this PR, merging it to trunk.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "showuon (via GitHub)" <gi...@apache.org>.
showuon commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1308148879


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -761,11 +784,385 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            // Build the leader epoch map by filtering the epochs that do not have any records.
+            NavigableMap<Integer, Long> epochWithOffsets = buildFilteredLeaderEpochMap(leaderEpochCache.epochWithOffsets());
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            long logStartOffset = log.logStartOffset();
+            long logEndOffset = log.logEndOffset();
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), logEndOffset, epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, logStartOffset);
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known
+            // to the leader. This will remove the unreferenced segments in the remote storage. This is needed for
+            // unclean leader election scenarios as the remote storage can have epochs earlier to the current leader's
+            // earliest leader epoch.
+            if (earliestEpochEntryOptional.isPresent()) {
+                EpochEntry earliestEpochEntry = earliestEpochEntryOptional.get();
+                Iterator<Integer> epochsToClean = remoteLeaderEpochs.stream().filter(x -> x < earliestEpochEntry.epoch).iterator();
+                while (epochsToClean.hasNext()) {
+                    int epoch = epochsToClean.next();
+                    Iterator<RemoteLogSegmentMetadata> segmentsToBeCleaned = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                    while (segmentsToBeCleaned.hasNext()) {
+                        if (isCancelled() || !isLeader()) {
+                            return;
+                        }
+                        // No need to update the log-start-offset even though the segment is deleted as these epochs/offsets are earlier to that value.
+                        remoteLogRetentionHandler.deleteLogSegmentsDueToLeaderEpochCacheTruncation(earliestEpochEntry, segmentsToBeCleaned.next());

Review Comment:
   @dopuskh3 , thanks for reporting this issue. I've created [KAFKA-15414](https://issues.apache.org/jira/browse/KAFKA-15414) for this issue. Let's discuss it in JIRA.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213107255


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;
+
+            // All the leader epochs in sorted order that exists in remote storage
+            List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            long remainingBreachedSize = totalSize - log.config().retentionSize;
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(remainingBreachedSize);
+
+            boolean checkTimestampRetention = retentionMs > -1;
+            long cleanupTs = time.milliseconds() - retentionMs;
+            boolean checkSizeRetention = log.config().retentionSize > -1;
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            Iterator<EpochEntry> epochEntryIterator = leaderEpochCache.epochEntries().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochEntryIterator.hasNext()) {
+                EpochEntry epochEntry = epochEntryIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epochEntry.epoch);

Review Comment:
   No, it is taken care of. When we remove a remote log segment, it also updates that entry in RLMM in synchronous manner. So, RLMM store will remove the entry from respective epoch states.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1213102668


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -618,6 +629,193 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            logger.debug("Updating $topicPartition with remoteLogStartOffset: {}", remoteLogStartOffset);
+            updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private long remainingBreachedSize = 0L;
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(long remainingBreachedSize) {
+                this.remainingBreachedSize = remainingBreachedSize;
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata, boolean checkSizeRetention) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size > 0
+                    if (checkSizeRetention && remainingBreachedSize > 0) {
+                        remainingBreachedSize -= x.segmentSizeInBytes();
+                        return remainingBreachedSize >= 0;
+                    } else return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment ${metadata.remoteLogSegmentId()} due to retention size " +
+                            "${log.config.retentionSize} breach. Log size after deletion will be " +
+                            "${remainingBreachedSize + log.config.retentionSize}.");
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> startOffset > x.endOffset());
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    logger.info("Deleted remote log segment {} due to log start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+                }
+
+                // No need to update the logStartOffset.
+                return isSegmentDeleted;
+            }
+
+            // There are two cases:
+            // 1) When there are offline partitions and a new replica with empty disk is brought as leader, then the
+            //    leader-epoch gets bumped but the log-start-offset gets truncated back to 0.
+            // 2) To remove the unreferenced segments.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment ${} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata,
+                                                               boolean checkTimestampRetention,
+                                                               long cleanupTs,
+                                                               long retentionMs)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> checkTimestampRetention && x.maxTimestampMs() <= cleanupTs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time " +
+                            "{}ms breach based on the largest record timestamp in the segment", metadata.remoteLogSegmentId(), retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled())
+                return;
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                return;
+            }
+
+            final long retentionMs = log.config().retentionMs;
+
+            // Iterate once
+            //  - to build the log size of segments with base-offset < local-log-start-offset
+            //  - to collect all the epochs of remote log segments
+            // These values can be cached and updated in RLMTask for this topic partition without computing in each
+            // iteration. But the logic can become little complex and need to cover different scenarios to avoid any
+            // leaks. We can have a followup to improve it by maintaining these values through both copying and deletion.
+            final Set<Integer> epochsSet = new HashSet<>();
+            long totalSizeEarlierToLocalLogStartOffset = 0L;
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+
+                if (segmentMetadata.endOffset() < log.localLogStartOffset()) {
+                    totalSizeEarlierToLocalLogStartOffset += segmentMetadata.segmentSizeInBytes();
+                }
+            }
+
+            // This is the total size of segments in local log that have their base-offset > local-log-start-offset
+            // and size of the segments in remote storage which have their end-offset < local-log-start-offset.
+            long totalSize = log.validLocalLogSegmentsSize() + totalSizeEarlierToLocalLogStartOffset;

Review Comment:
   `totalSizeEarlierToLocalLogStartOffset` computes only the log segments in remote storage beyond local-log-start-offset. The remaining local log segments size is computed separately. So, there will be no overlapping 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289577612


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1464,12 +1513,12 @@ class UnifiedLog(@volatile var logStartOffset: Long,
       }
     }
 
-    deleteOldSegments(shouldDelete, RetentionSizeBreach(this))
+    deleteOldSegments(shouldDelete, RetentionSizeBreach(this, remoteLogEnabled()))
   }
 
   private def deleteLogStartOffsetBreachedSegments(): Int = {
     def shouldDelete(segment: LogSegment, nextSegmentOpt: Option[LogSegment]): Boolean = {
-      nextSegmentOpt.exists(_.baseOffset <= logStartOffset)
+      nextSegmentOpt.exists(_.baseOffset <= localLogStartOffset())

Review Comment:
   Nice catch! Missed it while merging the conflicts.



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] satishd commented on a diff in pull request #13561: KAFKA-14888: Added remote log segments retention functionality based on time and size.

Posted by "satishd (via GitHub)" <gi...@apache.org>.
satishd commented on code in PR #13561:
URL: https://github.com/apache/kafka/pull/13561#discussion_r1289577043


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -696,11 +704,327 @@ public void run() {
             }
         }
 
+        public void handleLogStartOffsetUpdate(TopicPartition topicPartition, long remoteLogStartOffset) {
+            if (isLeader()) {
+                logger.debug("Updating {} with remoteLogStartOffset: {}", topicPartition, remoteLogStartOffset);
+                updateRemoteLogStartOffset.accept(topicPartition, remoteLogStartOffset);
+            }
+        }
+
+        class RemoteLogRetentionHandler {
+
+            private final Optional<RetentionSizeData> retentionSizeData;
+            private final Optional<RetentionTimeData> retentionTimeData;
+
+            private long remainingBreachedSize;
+
+            private OptionalLong logStartOffset = OptionalLong.empty();
+
+            public RemoteLogRetentionHandler(Optional<RetentionSizeData> retentionSizeData, Optional<RetentionTimeData> retentionTimeData) {
+                this.retentionSizeData = retentionSizeData;
+                this.retentionTimeData = retentionTimeData;
+                remainingBreachedSize = retentionSizeData.map(sizeData -> sizeData.remainingBreachedSize).orElse(0L);
+            }
+
+            private boolean deleteRetentionSizeBreachedSegments(RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionSizeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x -> {
+                    // Assumption that segments contain size >= 0
+                    if (remainingBreachedSize > 0) {
+                        long remainingBytes = remainingBreachedSize - x.segmentSizeInBytes();
+                        if (remainingBytes >= 0) {
+                            remainingBreachedSize = remainingBytes;
+                            return true;
+                        }
+                    }
+
+                    return false;
+                });
+                if (isSegmentDeleted) {
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention size {} breach. Log size after deletion will be {}.",
+                            metadata.remoteLogSegmentId(), retentionSizeData.get().retentionSize, remainingBreachedSize + retentionSizeData.get().retentionSize);
+                }
+                return isSegmentDeleted;
+            }
+
+            public boolean deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata metadata)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (!retentionTimeData.isPresent()) {
+                    return false;
+                }
+
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
+                        x -> x.maxTimestampMs() <= retentionTimeData.get().cleanupUntilMs);
+                if (isSegmentDeleted) {
+                    remainingBreachedSize = Math.max(0, remainingBreachedSize - metadata.segmentSizeInBytes());
+                    // It is fine to have logStartOffset as `metadata.endOffset() + 1` as the segment offset intervals
+                    // are ascending with in an epoch.
+                    logStartOffset = OptionalLong.of(metadata.endOffset() + 1);
+                    logger.info("Deleted remote log segment {} due to retention time {}ms breach based on the largest record timestamp in the segment",
+                            metadata.remoteLogSegmentId(), retentionTimeData.get().retentionMs);
+                }
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long startOffset)
+                    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);
+                }
+
+                return isSegmentDeleted;
+            }
+
+            // It removes the segments beyond the current leader's earliest epoch. Those segments are considered as
+            // unreferenced because they are not part of the current leader epoch lineage.
+            private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry, RemoteLogSegmentMetadata metadata) throws RemoteStorageException, ExecutionException, InterruptedException {
+                boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x ->
+                        x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch < earliestEpochEntry.epoch));
+                if (isSegmentDeleted) {
+                    logger.info("Deleted remote log segment {} due to leader epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and segmentEpochs: {}",
+                            metadata.remoteLogSegmentId(), earliestEpochEntry, metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
+                }
+
+                // No need to update the log-start-offset as these epochs/offsets are earlier to that value.
+                return isSegmentDeleted;
+            }
+
+            private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
+                    throws RemoteStorageException, ExecutionException, InterruptedException {
+                if (predicate.test(segmentMetadata)) {
+                    logger.info("Deleting remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    // Publish delete segment started event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, brokerId)).get();
+
+                    // Delete the segment in remote storage.
+                    remoteLogStorageManager.deleteLogSegmentData(segmentMetadata);
+
+                    // Publish delete segment finished event.
+                    remoteLogMetadataManager.updateRemoteLogSegmentMetadata(
+                            new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), time.milliseconds(),
+                                    segmentMetadata.customMetadata(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, brokerId)).get();
+                    logger.info("Deleted remote log segment {}", segmentMetadata.remoteLogSegmentId());
+                    return true;
+                }
+
+                return false;
+            }
+
+        }
+
+        private void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionException, InterruptedException {
+            if (isCancelled() || !isLeader()) {
+                logger.info("Returning from remote log segments cleanup as the task state is changed");
+                return;
+            }
+
+            // Cleanup remote log segments and update the log start offset if applicable.
+            final Iterator<RemoteLogSegmentMetadata> segmentMetadataIter = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition);
+            if (!segmentMetadataIter.hasNext()) {
+                logger.debug("No remote log segments available on remote storage for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Optional<UnifiedLog> logOptional = fetchLog.apply(topicIdPartition.topicPartition());
+            if (!logOptional.isPresent()) {
+                logger.debug("No UnifiedLog instance available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final UnifiedLog log = logOptional.get();
+            final Option<LeaderEpochFileCache> leaderEpochCacheOption = log.leaderEpochCache();
+            if (leaderEpochCacheOption.isEmpty()) {
+                logger.debug("No leader epoch cache available for partition: {}", topicIdPartition);
+                return;
+            }
+
+            final Set<Integer> epochsSet = new HashSet<>();
+            // Good to have an API from RLMM to get all the remote leader epochs of all the segments of a partition
+            // instead of going through all the segments and building it here.
+            while (segmentMetadataIter.hasNext()) {
+                RemoteLogSegmentMetadata segmentMetadata = segmentMetadataIter.next();
+                epochsSet.addAll(segmentMetadata.segmentLeaderEpochs().keySet());
+            }
+
+            // All the leader epochs in sorted order that exists in remote storage
+            final List<Integer> remoteLeaderEpochs = new ArrayList<>(epochsSet);
+            Collections.sort(remoteLeaderEpochs);
+
+            LeaderEpochFileCache leaderEpochCache = leaderEpochCacheOption.get();
+            NavigableMap<Integer, Long> epochWithOffsets = leaderEpochCache.epochWithOffsets();
+            Optional<EpochEntry> earliestEpochEntryOptional = leaderEpochCache.earliestEntry();
+
+            Optional<RetentionSizeData> retentionSizeData = buildRetentionSizeData(log.config().retentionSize,
+                    log.onlyLocalLogSegmentsSize(), log.logEndOffset(), epochWithOffsets);
+            Optional<RetentionTimeData> retentionTimeData = buildRetentionTimeData(log.config().retentionMs);
+
+            RemoteLogRetentionHandler remoteLogRetentionHandler = new RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
+            Iterator<Integer> epochIterator = epochWithOffsets.navigableKeySet().iterator();
+            boolean isSegmentDeleted = true;
+            while (isSegmentDeleted && epochIterator.hasNext()) {
+                Integer epoch = epochIterator.next();
+                Iterator<RemoteLogSegmentMetadata> segmentsIterator = remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
+                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                    if (isCancelled() || !isLeader()) {
+                        logger.info("Returning from remote log segments cleanup for the remaining segments as the task state is changed.");
+                        return;
+                    }
+                    RemoteLogSegmentMetadata metadata = segmentsIterator.next();
+
+                    // check whether the segment contains the required epoch range with in the current leader epoch lineage.
+                    if (isRemoteSegmentWithinLeaderEpochs(metadata, log.logEndOffset(), epochWithOffsets)) {
+                        isSegmentDeleted =
+                                remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
+                                        remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, log.logStartOffset());
+                    }
+                }
+            }
+
+            // Remove the remote log segments whose segment-leader-epochs are less than the earliest-epoch known

Review Comment:
   This covers scenarios where unclean leader election happens and the remote storage contains segments that are earlier to the current leader's leader-epoch-lineage. 
   
   For ex:
   
   The current leader has the current leader-epoch-cache.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        3              700
        4              780
        6              900
        7              990 					
   -----------------------------
   ```
   But the earlier broker which got replaced with a new broker which has the current leader's leader-epoch lineage.
   ```
   -----------------------------
   leader-epoch | start-offset |
   -----------------------------
        0              0
        1              180
        2              400
   -----------------------------
   ```
   But these segments did not expire retention and they were not deleted in the remote storage. But these leader epochs are not there in the current leader's leader epoch as it was chosen with unclean leader election. In this case, we need to remove the segments, that exist beyond the current leader epoch lineage. Otherwise, they will never be cleaned up and will continue to accumulate in remote storage.
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org