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


##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsDueToLogStartOffsetBreachTest.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.Arrays;
+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;
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+
+public final class DeleteSegmentsDueToLogStartOffsetBreachTest extends 
TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 2;
+    }
+
+    @Override
+    public Properties overridingProps() {
+        Properties props = super.overridingProps();
+        // configure infinite bytes retention
+        props.put(KafkaConfig.LogRetentionBytesProp(), "-1");
+        return props;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final Integer broker0 = 0;
+        final Integer broker1 = 1;
+        final String topicA = "topicA";
+        final Integer p0 = 0;
+        final Integer partitionCount = 1;
+        final Integer replicationFactor = 2;
+        final Integer maxBatchCountPerSegment = 2;
+        final Map<Integer, List<Integer>> replicaAssignment = 
mkMap(mkEntry(p0, Arrays.asList(broker0, broker1)));
+        final boolean enableRemoteLogStorage = true;
+        final int beginEpoch = 0;
+        final long startOffset = 3;
+        final long beforeOffset = 3L;
+
+        // Create topicA with 1 partition and 2 RF
+        builder.createTopic(topicA, partitionCount, replicationFactor, 
maxBatchCountPerSegment, replicaAssignment,
+                        enableRemoteLogStorage)
+                // produce events to partition 0 and expect 2 segments to be 
offloaded
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new 
KeyValueSpec("k0", "v0"),
+                        new KeyValueSpec("k1", "v1"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 2, new 
KeyValueSpec("k2", "v2"),
+                        new KeyValueSpec("k3", "v3"))
+                .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 4L)
+                .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new 
KeyValueSpec("k1", "v1"),
+                        new KeyValueSpec("k2", "v2"), new KeyValueSpec("k3", 
"v3"), new KeyValueSpec("k4", "v4"))
+                // Use DELETE_RECORDS API to delete the records upto offset 3 
and expect one remote segment to be deleted
+                .expectDeletionInRemoteStorage(broker0, topicA, p0, 
DELETE_SEGMENT, 1)
+                .deleteRecords(topicA, p0, beforeOffset)
+                // expect that the leader epoch checkpoint is updated
+                // Comment out this line if it's FLAKY since the leader-epoch 
is not deterministic in ZK mode.
+                .expectLeaderEpochCheckpoint(broker0, topicA, p0, beginEpoch, 
startOffset)
+                // consume from the offset-3 of the topic to read data from 
local and remote storage
+                .expectFetchFromTieredStorage(broker0, topicA, p0, 1)
+                .consume(topicA, p0, 3L, 2, 1)
+
+                // switch leader to change the leader-epoch from 0 to 1
+                .expectLeader(topicA, p0, broker1, true)
+                // produce some more messages and move the log-start-offset 
such that earliest-epoch changes from 0 to 1
+                .expectSegmentToBeOffloaded(broker1, topicA, p0, 4, new 
KeyValueSpec("k4", "v4"),
+                        new KeyValueSpec("k5", "v5"))
+                .expectSegmentToBeOffloaded(broker1, topicA, p0, 6, new 
KeyValueSpec("k6", "v6"),
+                        new KeyValueSpec("k7", "v7"))
+                .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 8L)
+                .produce(topicA, p0, new KeyValueSpec("k5", "v5"), new 
KeyValueSpec("k6", "v6"),

Review Comment:
   Good to add a check for the updated leader epochs with this leader.



##########
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());

Review Comment:
   Here, we may be removing the segments of different leader epoch chain 
containing the offsets < start-offset. Better to have the respective checks 
before deleting the segment. 



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1066,50 +1066,53 @@ public static boolean 
isRemoteSegmentWithinLeaderEpochs(RemoteLogSegmentMetadata
         // Check for out of bound epochs between segment epochs and current 
leader epochs.
         Integer segmentFirstEpoch = segmentLeaderEpochs.firstKey();
         Integer segmentLastEpoch = segmentLeaderEpochs.lastKey();
+
+        // FIXME: We have to remove the below check too for `DELETE_RECORDS` 
API to work properly.
         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);
             return false;
         }
 
         for (Map.Entry<Integer, Long> entry : segmentLeaderEpochs.entrySet()) {
             int epoch = entry.getKey();
-            long offset = entry.getValue();
+//            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);
+                LOGGER.debug("Segment {} epoch {} is not within the leader 
epoch lineage. " +
+                                "Remote segment epochs: {} and partition 
leader epochs: {}",
+                        segmentMetadata.remoteLogSegmentId(), epoch, 
segmentLeaderEpochs, leaderEpochs);
                 return false;
             }
 
             // Segment's first epoch's offset should 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 less than leader epoch's offset {}.",
-                        segmentMetadata.topicIdPartition(), 
segmentMetadata.remoteLogSegmentId(), epoch, leaderEpochs.get(epoch));
-                return false;
-            }
+//            if (epoch == segmentFirstEpoch && offset < 
leaderEpochs.get(epoch)) {
+//                LOGGER.debug("Segment {} first epoch {} offset is less than 
leader epoch offset {}.",
+//                        segmentMetadata.remoteLogSegmentId(), epoch, 
leaderEpochs.get(epoch));
+//                return false;
+//            }
 
             // Segment's end offset should be less than or equal to the 
respective leader epoch's offset.
             if (epoch == segmentLastEpoch) {
                 Map.Entry<Integer, Long> nextEntry = 
leaderEpochs.higherEntry(epoch);
                 if (nextEntry != null && segmentEndOffset > 
nextEntry.getValue() - 1) {
-                    LOGGER.debug("[{}]  Remote segment {}'s end offset {} is 
more than leader epoch's offset {}.",
-                            segmentMetadata.topicIdPartition(), 
segmentMetadata.remoteLogSegmentId(), segmentEndOffset, nextEntry.getValue() - 
1);
+                    LOGGER.debug("Segment {} end offset {} is more than leader 
epoch offset {}.",
+                            segmentMetadata.remoteLogSegmentId(), 
segmentEndOffset, nextEntry.getValue() - 1);
                     return false;
                 }
             }
 
             // Next segment epoch entry and next leader epoch entry should be 
same to ensure that the segment's epoch
             // is within the leader epoch lineage.
             if (epoch != segmentLastEpoch && 
!leaderEpochs.higherEntry(epoch).equals(segmentLeaderEpochs.higherEntry(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);
+                LOGGER.debug("Segment {} epoch {} is not within the leader 
epoch lineage. " +

Review Comment:
   Good catch avoiding partition printed twice.



##########
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:
   Should not `remainingBreachedSize` updated only when retention by size is 
enabled? I agree we should always log the message if the segment is deleted. 
One way to do that is like below.
   ```
   if (isSegmentDeleted) {
       if (retentionSizeData.isPresent()) {
           remainingBreachedSize = Math.max(0, remainingBreachedSize - 
metadata.segmentSizeInBytes());
       }
       logger.info("Deleted remote log segment {} due to log start offset {} 
breach", metadata.remoteLogSegmentId(), startOffset);
   }
   ```



-- 
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