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


##########
core/src/main/scala/kafka/log/remote/RemoteLogManager.scala:
##########
@@ -272,15 +364,270 @@ class RemoteLogManager(rlmConfig: RemoteLogManagerConfig,
     None
   }
 
+  trait CancellableRunnable extends Runnable {
+    @volatile private var cancelled = false
+
+    def cancel(): Unit = {
+      cancelled = true
+    }
+
+    def isCancelled(): Boolean = {
+      cancelled
+    }
+  }
+
+  /**
+   * Returns the leader epoch checkpoint by truncating with the given 
start[exclusive] and end[inclusive] offset
+   *
+   * @param log         The actual log from where to take the leader-epoch 
checkpoint
+   * @param startOffset The start offset of the checkpoint file (exclusive in 
the truncation).
+   *                    If start offset is 6, then it will retain an entry at 
offset 6.
+   * @param endOffset   The end offset of the checkpoint file (inclusive in 
the truncation)
+   *                    If end offset is 100, then it will remove the entries 
greater than or equal to 100.
+   * @return the truncated leader epoch checkpoint
+   */
+  private[remote] def getLeaderEpochCheckpoint(log: UnifiedLog, startOffset: 
Long, endOffset: Long): InMemoryLeaderEpochCheckpoint = {
+    val checkpoint = new InMemoryLeaderEpochCheckpoint()
+    log.leaderEpochCache
+      .map(cache => cache.writeTo(checkpoint))
+      .foreach { x =>
+        if (startOffset >= 0) {
+          x.truncateFromStart(startOffset)
+        }
+        x.truncateFromEnd(endOffset)
+      }
+    checkpoint
+  }
+
+  private[remote] class RLMTask(tpId: TopicIdPartition) extends 
CancellableRunnable with Logging {
+    this.logIdent = s"[RemoteLogManager=$brokerId partition=$tpId] "
+    @volatile private var leaderEpoch: Int = -1
+
+    private def isLeader(): Boolean = leaderEpoch >= 0
+
+    // The readOffset is None initially for a new leader RLMTask, and needs to 
be fetched inside the task's run() method.
+    @volatile private var copiedOffsetOption: Option[Long] = None
+
+    def convertToLeader(leaderEpochVal: Int): Unit = {
+      if (leaderEpochVal < 0) {
+        throw new KafkaException(s"leaderEpoch value for topic partition $tpId 
can not be negative")
+      }
+      if (this.leaderEpoch != leaderEpochVal) {
+        leaderEpoch = leaderEpochVal
+      }
+      // Reset readOffset, so that it is set in next run of RLMTask
+      copiedOffsetOption = None
+    }
+
+    def convertToFollower(): Unit = {
+      leaderEpoch = -1
+    }
+
+    def handleCopyLogSegmentsToRemote(): Unit = {
+      if (isCancelled())
+        return
+
+      def maybeUpdateReadOffset(): Unit = {
+        if (copiedOffsetOption.isEmpty) {
+          info(s"Find the highest remote offset for partition: $tpId after 
becoming leader, leaderEpoch: $leaderEpoch")
+
+          // This is found by traversing from the latest leader epoch from 
leader epoch history and find the highest offset
+          // of a segment with that epoch copied into remote storage. If it 
can not find an entry then it checks for the
+          // previous leader epoch till it finds an entry, If there are no 
entries till the earliest leader epoch in leader
+          // epoch cache then it starts copying the segments from the earliest 
epoch entry’s offset.
+          copiedOffsetOption = Some(findHighestRemoteOffset(tpId))
+        }
+      }
+
+      try {
+        maybeUpdateReadOffset()
+        val copiedOffset = copiedOffsetOption.get
+        fetchLog(tpId.topicPartition()).foreach { log =>
+          // LSO indicates the offset below are ready to be 
consumed(high-watermark or committed)
+          val lso = log.lastStableOffset
+          if (lso < 0) {
+            warn(s"lastStableOffset for partition $tpId is $lso, which should 
not be negative.")
+          } else if (lso > 0 && copiedOffset < lso) {
+            // Copy segments only till the min of high-watermark or 
stable-offset as remote storage should contain
+            // only committed/acked messages
+            val toOffset = lso

Review Comment:
   LastStableOffset already represents that, will update the javadoc comment.



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