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


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -954,24 +953,29 @@ private void cleanupExpiredRemoteLogSegments() throws 
RemoteStorageException, Ex
 
             RemoteLogRetentionHandler remoteLogRetentionHandler = new 
RemoteLogRetentionHandler(retentionSizeData, retentionTimeData);
             Iterator<Integer> epochIterator = 
epochWithOffsets.navigableKeySet().iterator();
-            boolean isSegmentDeleted = true;
-            while (isSegmentDeleted && epochIterator.hasNext()) {
+            boolean canProcess = true;
+            while (canProcess && epochIterator.hasNext()) {
                 Integer epoch = epochIterator.next();
                 Iterator<RemoteLogSegmentMetadata> segmentsIterator = 
remoteLogMetadataManager.listRemoteLogSegments(topicIdPartition, epoch);
-                while (isSegmentDeleted && segmentsIterator.hasNext()) {
+                while (canProcess && 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)) {
+                    boolean isValidSegment = 
isRemoteSegmentWithinLeaderEpochs(metadata, logEndOffset, epochWithOffsets);
+                    boolean isSegmentDeleted = false;
+                    if (isValidSegment) {
                         isSegmentDeleted =
                                 
remoteLogRetentionHandler.deleteRetentionTimeBreachedSegments(metadata) ||
-                                        
remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata) ||
-                                        
remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, 
logStartOffset);
+                                        
remoteLogRetentionHandler.deleteRetentionSizeBreachedSegments(metadata);
+                    }
+                    if (!isSegmentDeleted) {
+                        isSegmentDeleted = 
remoteLogRetentionHandler.deleteLogStartOffsetBreachedSegments(metadata, 
logStartOffset);

Review Comment:
   Why does the `logStartOffestBreach` doesn't need to care if 
`isRemoteSegmentWithinLeaderEpoch`? 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1070,8 +1074,9 @@ public static boolean 
isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata
         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.topicIdPartition(), 
segmentMetadata.remoteLogSegmentId(), segmentLeaderEpochs, leaderEpochs);
+            LOGGER.debug("Segment {} is not within the partition leader epoch 
lineage. " +
+                            "Remote segment epochs: {} and partition leader 
epochs: {}",
+                    segmentMetadata.remoteLogSegmentId(), segmentLeaderEpochs, 
leaderEpochs);

Review Comment:
   Why we remove the partition data in the log? log context already contains it?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -856,11 +856,10 @@ public boolean 
deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata meta
             private boolean 
deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long 
startOffset)
                     throws RemoteStorageException, ExecutionException, 
InterruptedException {
                 boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x 
-> startOffset > x.endOffset());
-                if (isSegmentDeleted && retentionSizeData.isPresent()) {
+                if (isSegmentDeleted) {

Review Comment:
   Ah, it's not related to retentionSize config. Nice catch.



##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.tiered.storage.integration;
+
+import kafka.server.KafkaConfig;
+import org.apache.kafka.tiered.storage.TieredStorageTestBuilder;
+import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import static 
org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_SEGMENT;
+
+public final class DeleteSegmentsDueToLogStartOffsetBreachTest extends 
TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    public Properties overridingProps() {
+        Properties props = super.overridingProps();
+        // configure infinite bytes retention
+        props.put(KafkaConfig.LogRetentionBytesProp(), "-1");

Review Comment:
   Please make the comment much clear. It will confuse readers that the `-1` 
config is a required config for this test.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1070,8 +1074,9 @@ public static boolean 
isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata
         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.topicIdPartition(), 
segmentMetadata.remoteLogSegmentId(), segmentLeaderEpochs, leaderEpochs);
+            LOGGER.debug("Segment {} is not within the partition leader epoch 
lineage. " +
+                            "Remote segment epochs: {} and partition leader 
epochs: {}",
+                    segmentMetadata.remoteLogSegmentId(), segmentLeaderEpochs, 
leaderEpochs);

Review Comment:
   OK, saw it below. 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-unsubscr...@kafka.apache.org

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

Reply via email to