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


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1424,25 +1425,71 @@ 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 = {
-      val upperBoundOffset = 
nextSegmentOpt.map(_.baseOffset).getOrElse(localLog.logEndOffset)
-
-      // Check not to delete segments which are not yet copied to tiered 
storage if remote log is enabled.
-      (!remoteLogEnabled() || (upperBoundOffset > 0 && upperBoundOffset - 1 <= 
highestOffsetInRemoteStorage)) &&
-        // We don't delete segments with offsets at or beyond the high 
watermark to ensure that the log start
-        // offset can never exceed it.
-        highWatermark >= upperBoundOffset &&
-        predicate(segment, nextSegmentOpt)
-    }
     lock synchronized {
-      val deletable = localLog.deletableSegments(shouldDelete)
+      val deletable = deletableSegments(predicate)
       if (deletable.nonEmpty)
         deleteSegments(deletable, reason)
       else
         0
     }
   }
 
+  /**
+   * Find segments starting from the oldest until the user-supplied predicate 
is false.
+   * A final segment that is empty will never be returned.
+   *
+   * @param predicate A function that takes in a candidate log segment, the 
next higher segment
+   *                  (if there is one). It returns true iff the segment is 
deletable.
+   * @return the segments ready to be deleted
+   */
+  private[log] def deletableSegments(predicate: (LogSegment, 
Option[LogSegment]) => Boolean): Iterable[LogSegment] = {
+    def isSegmentEligibleForDeletion(upperBoundOffset: Long): Boolean = {
+      // Segments are eligible for deletion when:
+      //    1. they are uploaded to the remote storage
+      if (remoteLogEnabled()) {
+        upperBoundOffset > 0 && upperBoundOffset - 1 <= 
highestOffsetInRemoteStorage
+      } else {
+        true
+      }
+    }
+
+    if (localLog.segments.isEmpty) {
+      Seq.empty
+    } else {
+      val deletable = ArrayBuffer.empty[LogSegment]
+      val segmentsIterator = localLog.segments.values.iterator
+      var segmentOpt = nextOption(segmentsIterator)
+      var shouldRoll = false
+      while (segmentOpt.isDefined) {
+        val segment = segmentOpt.get
+        val nextSegmentOpt = nextOption(segmentsIterator)
+        val isLastSegmentAndEmpty = nextSegmentOpt.isEmpty && segment.size == 0
+        val upperBoundOffset = if (nextSegmentOpt.nonEmpty) 
nextSegmentOpt.get.baseOffset() else logEndOffset
+        // We don't delete segments with offsets at or beyond the high 
watermark to ensure that the log start
+        // offset can never exceed it.
+        val predicateResult = highWatermark >= upperBoundOffset && 
predicate(segment, nextSegmentOpt)
+
+        // Roll the active segment when it breaches the configured retention 
policy. The rolled segment will be
+        // eligible for deletion and gets removed in the next iteration.
+        if (predicateResult && !isLastSegmentAndEmpty && remoteLogEnabled() && 
nextSegmentOpt.isEmpty) {

Review Comment:
   Can we have the below condition which is easy to understand or some thing 
better?
   ```
   predicateResult && remoteLogEnabled() &&
   nextSegmentOpt.isEmpty  && segment.size > 0 // active segment is not empty
   ```
   
   Another way is to keep the last segment's predicate result and its size and 
do the check after the while loop. 



##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/RollAndOffloadActiveSegmentTest.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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 org.apache.kafka.common.config.TopicConfig;
+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.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Test to verify that the active segment is rolled and uploaded to remote 
storage when the segment breaches the
+ * local log retention policy.
+ */
+public class RollAndOffloadActiveSegmentTest extends TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final Integer broker0 = 0;
+        final String topicA = "topicA";
+        final Integer p0 = 0;
+        final Integer partitionCount = 1;
+        final Integer replicationFactor = 1;
+        final Integer maxBatchCountPerSegment = 1;
+        final Map<Integer, List<Integer>> replicaAssignment = null;
+        final boolean enableRemoteLogStorage = true;
+
+        // Create topicA with 1 partition, 1 RF and enabled with remote 
storage.
+        builder.createTopic(topicA, partitionCount, replicationFactor, 
maxBatchCountPerSegment, replicaAssignment,
+                        enableRemoteLogStorage)
+                // update the topic config such that it triggers the rolling 
of the active segment
+                .updateTopicConfig(topicA, configsToBeAdded(), 
Collections.emptyList())
+                // produce events to partition 0 and expect all the 4 segments 
to be offloaded
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new 
KeyValueSpec("k0", "v0"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new 
KeyValueSpec("k1", "v1"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 2, new 
KeyValueSpec("k2", "v2"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 3, 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"))
+                // consume from the beginning of the topic to read data from 
local and remote storage
+                .expectFetchFromTieredStorage(broker0, topicA, p0, 4)
+                .consume(topicA, p0, 0L, 4, 4);

Review Comment:
   `OffsetSpec` updates are tracked under 
https://issues.apache.org/jira/browse/KAFKA-15857
   
   Filed https://issues.apache.org/jira/browse/KAFKA-15864 to track the 
targeted tests.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to