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


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1424,25 +1425,70 @@ 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 && remoteLogEnabled() && nextSegmentOpt.isEmpty && 
segment.size > 0) {
+          shouldRoll = true

Review Comment:
   > We have checks to allow only the passive segments to be uploaded, so the 
active segment never gets removed at all even if breaches the retention time.
   
   Hmm, the above description of the PR doesn't seem correct. Before this PR, 
we will consider the deletion of the active segment as long as it's not empty. 
If the active segment is meets the deletion criteria, we will delete it and 
automatically roll a new empty segment. So, why do we need to explicitly roll 
here? Also, why do we only do that when remote log 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-unsubscr...@kafka.apache.org

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

Reply via email to