chia7712 commented on code in PR #19630:
URL: https://github.com/apache/kafka/pull/19630#discussion_r2085105937


##########
core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala:
##########
@@ -154,40 +157,40 @@ class RemoteLeaderEndPoint(logPrefix: String,
           val tp = new TopicPartition(offsetForLeaderTopicResult.topic, 
offsetForLeaderPartitionResult.partition)
           tp -> offsetForLeaderPartitionResult
         }
-      }.toMap
+      }.toMap.asJava
     } catch {
       case t: Throwable =>
         warn(s"Error when sending leader epoch request for $partitions", t)
 
         // if we get any unexpected exception, mark all partitions with an 
error
         val error = Errors.forException(t)
-        partitions.map { case (tp, _) =>
+        tmpPartitions.map { case (tp, _) =>
           tp -> new EpochEndOffset()
             .setPartition(tp.partition)
             .setErrorCode(error.code)
-        }
+        }.asJava
     }
   }
 
-  override def buildFetch(partitions: Map[TopicPartition, 
PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
+  override def buildFetch(partitions: java.util.Map[TopicPartition, 
PartitionFetchState]): ResultWithPartitions[java.util.Optional[ReplicaFetch]] = 
{
     val partitionsWithError = mutable.Set[TopicPartition]()
-
+    val tmpPartitions = partitions.asScala.toMap
     val builder = fetchSessionHandler.newBuilder(partitions.size, false)
-    partitions.foreachEntry { (topicPartition, fetchState) =>
+    tmpPartitions.foreachEntry { (topicPartition, fetchState) =>

Review Comment:
   ```
   partitions.forEach
   ```



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -204,11 +210,25 @@ abstract class AbstractFetcherThread(name: String,
    * occur during truncation.
    */
   private def truncateToEpochEndOffsets(latestEpochsForPartitions: 
Map[TopicPartition, EpochData]): Unit = {
-    val endOffsets = leader.fetchEpochEndOffsets(latestEpochsForPartitions)
-    //Ensure we hold a lock during truncation.
+
+    val partitionsMap = new java.util.HashMap[TopicPartition, 
OffsetForLeaderPartition]()
+
+    latestEpochsForPartitions.asJava.forEach { case (tp, epochData) =>
+      // Create a new OffsetForLeaderPartition from EpochData
+      val offsetForLeaderPartition = new OffsetForLeaderPartition()
+        .setPartition(tp.partition())
+        .setCurrentLeaderEpoch(epochData.currentLeaderEpoch)
+        .setLeaderEpoch(epochData.leaderEpoch)
+
+      partitionsMap.put(tp, offsetForLeaderPartition)
+    }
+
+    val endOffsets = leader.fetchEpochEndOffsets(partitionsMap)

Review Comment:
   ```
   val endOffsets = 
leader.fetchEpochEndOffsets(latestEpochsForPartitions.asJava)
   ```



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -284,7 +304,7 @@ abstract class AbstractFetcherThread(name: String,
       }
     }
 
-    ResultWithPartitions(fetchOffsets, partitionsWithError)
+    new ResultWithPartitions[Map[TopicPartition, 
OffsetTruncationState]](fetchOffsets, new 
java.util.HashSet[TopicPartition](partitionsWithError.asJava))

Review Comment:
   ```
   new ResultWithPartitions(fetchOffsets, partitionsWithError.asJava)
   ```



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -204,11 +210,25 @@ abstract class AbstractFetcherThread(name: String,
    * occur during truncation.
    */
   private def truncateToEpochEndOffsets(latestEpochsForPartitions: 
Map[TopicPartition, EpochData]): Unit = {
-    val endOffsets = leader.fetchEpochEndOffsets(latestEpochsForPartitions)
-    //Ensure we hold a lock during truncation.
+
+    val partitionsMap = new java.util.HashMap[TopicPartition, 
OffsetForLeaderPartition]()
+
+    latestEpochsForPartitions.asJava.forEach { case (tp, epochData) =>
+      // Create a new OffsetForLeaderPartition from EpochData
+      val offsetForLeaderPartition = new OffsetForLeaderPartition()
+        .setPartition(tp.partition())
+        .setCurrentLeaderEpoch(epochData.currentLeaderEpoch)
+        .setLeaderEpoch(epochData.leaderEpoch)
+
+      partitionsMap.put(tp, offsetForLeaderPartition)
+    }
+
+    val endOffsets = leader.fetchEpochEndOffsets(partitionsMap)
+
+    // Ensure we hold a lock during truncation
     inLock(partitionMapLock) {
-      //Check no leadership and no leader epoch changes happened whilst we 
were unlocked, fetching epochs

Review Comment:
   please keep this comment



##########
server/src/main/java/org/apache/kafka/server/PartitionFetchState.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.server;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.utils.Time;
+
+import java.util.Optional;
+
+/**
+ * Class to keep partition offset and its state (truncatingLog, delayed)
+ * This represents a partition as being either:
+ * (1) Truncating its log, for example, having recently become a follower
+ * (2) Delayed, for example, due to an error, where we subsequently back off a 
bit
+ * (3) ReadyForFetch, the active state where the thread is actively fetching 
data.
+ */
+public class PartitionFetchState {

Review Comment:
   this can be record class, right?



##########
core/src/main/java/kafka/server/TierStateMachine.java:
##########
@@ -136,12 +136,13 @@ private OffsetForLeaderEpochResponseData.EpochEndOffset 
fetchEarlierEpochEndOffs
         // Find the end-offset for the epoch earlier to the given epoch from 
the leader
         Map<TopicPartition, 
OffsetForLeaderEpochRequestData.OffsetForLeaderPartition> partitionsWithEpochs 
= new HashMap<>();
         partitionsWithEpochs.put(partition, new 
OffsetForLeaderEpochRequestData.OffsetForLeaderPartition().setPartition(partition.partition()).setCurrentLeaderEpoch(currentLeaderEpoch).setLeaderEpoch(previousEpoch));
-        Option<OffsetForLeaderEpochResponseData.EpochEndOffset> 
maybeEpochEndOffset = 
leader.fetchEpochEndOffsets(CollectionConverters.asScala(partitionsWithEpochs)).get(partition);
-        if (maybeEpochEndOffset.isEmpty()) {
+        var endOffsets = leader.fetchEpochEndOffsets(partitionsWithEpochs);

Review Comment:
   ```
   var epochEndOffset = 
leader.fetchEpochEndOffsets(partitionsWithEpochs).get(partition);
   ```



##########
core/src/test/scala/unit/kafka/server/MockLeaderEndPoint.scala:
##########
@@ -136,39 +135,41 @@ class MockLeaderEndPoint(sourceBroker: BrokerEndPoint = 
new BrokerEndPoint(1, "l
     new OffsetAndEpoch(leaderState.localLogStartOffset, 
leaderState.leaderEpoch)
   }
 
-  override def fetchEpochEndOffsets(partitions: Map[TopicPartition, 
EpochData]): Map[TopicPartition, EpochEndOffset] = {
+  override def fetchEpochEndOffsets(partitions: java.util.Map[TopicPartition, 
OffsetForLeaderEpochRequestData.OffsetForLeaderPartition]): 
java.util.Map[TopicPartition, EpochEndOffset] = {
     val endOffsets = mutable.Map[TopicPartition, EpochEndOffset]()
-    partitions.foreachEntry { (partition, epochData) =>
+    val tmpPartitions = partitions.asScala
+    tmpPartitions.foreachEntry { (partition, epochData) =>
       assert(partition.partition == epochData.partition,
         "Partition must be consistent between TopicPartition and EpochData")
       val leaderState = leaderPartitionState(partition)
       val epochEndOffset = lookupEndOffsetForEpoch(partition, epochData, 
leaderState)
       endOffsets.put(partition, epochEndOffset)
     }
-    endOffsets
+    endOffsets.asJava
   }
 
-  override def buildFetch(partitionMap: Map[TopicPartition, 
PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = {
+  override def buildFetch(partitions: java.util.Map[TopicPartition, 
PartitionFetchState]): ResultWithPartitions[java.util.Optional[ReplicaFetch]] = 
{
     val fetchData = mutable.Map.empty[TopicPartition, 
FetchRequest.PartitionData]
-    partitionMap.foreach { case (partition, state) =>
+    val tmpPartitions = partitions.asScala.toMap
+    tmpPartitions.foreach { case (partition, state) =>
       if (state.isReadyForFetch) {
         val replicaState = 
replicaPartitionStateCallback(partition).getOrElse(throw new 
IllegalArgumentException(s"Unknown partition $partition"))
         val lastFetchedEpoch = if (isTruncationOnFetchSupported)
           state.lastFetchedEpoch
         else
           Optional.empty[Integer]
         fetchData.put(partition,
-          new 
FetchRequest.PartitionData(state.topicId.getOrElse(Uuid.ZERO_UUID), 
state.fetchOffset, replicaState.logStartOffset,
+          new FetchRequest.PartitionData(state.topicId.orElse(Uuid.ZERO_UUID), 
state.fetchOffset, replicaState.logStartOffset,
             1024 * 1024, Optional.of[Integer](state.currentLeaderEpoch), 
lastFetchedEpoch))
       }
     }
     val fetchRequest = FetchRequest.Builder.forReplica(version, replicaId, 1, 
0, 1, fetchData.asJava)
     val fetchRequestOpt =
       if (fetchData.isEmpty)
-        None
+        java.util.Optional.empty[ReplicaFetch]()
       else
-        Some(ReplicaFetch(fetchData.asJava, fetchRequest))
-    ResultWithPartitions(fetchRequestOpt, Set.empty)
+        Optional.of(new ReplicaFetch(fetchData.asJava, fetchRequest))
+    new 
ResultWithPartitions[java.util.Optional[ReplicaFetch]](fetchRequestOpt, 
java.util.Collections.emptySet[TopicPartition]())

Review Comment:
   ```
   new ResultWithPartitions(fetchRequestOpt, java.util.Collections.emptySet())
   ```



##########
core/src/main/scala/kafka/server/AbstractFetcherThread.scala:
##########
@@ -779,7 +799,7 @@ abstract class AbstractFetcherThread(name: String,
 
       // TODO: use fetchTierStateMachine.maybeAdvanceState when implementing 
async tiering logic in KAFKA-13560
 
-      fetcherLagStats.getAndMaybePut(topicPartition).lag = 
newFetchState.lag.getOrElse(0)
+      fetcherLagStats.getAndMaybePut(topicPartition).lag = if 
(newFetchState.lag.isPresent) newFetchState.lag.get() else 0L

Review Comment:
   ```
   newFetchState.lag.orElse(0L)
   ```



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