chia7712 commented on code in PR #19630: URL: https://github.com/apache/kafka/pull/19630#discussion_r2083219505
########## core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala: ########## @@ -216,8 +221,8 @@ class LocalLeaderEndPoint(sourceBroker: BrokerEndPoint, partitionsWithError += topicPartition } - val fetchRequestOpt = if (requestMap.isEmpty) { - None + val fetchRequestOpt: Optional[ReplicaFetch] = if (requestMap.isEmpty) { Review Comment: we don't need to declare the type, right? ########## core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala: ########## @@ -84,10 +86,10 @@ class RemoteLeaderEndPoint(logPrefix: String, if (fetchResponse.error == Errors.FETCH_SESSION_TOPIC_ID_ERROR) { throw Errors.forCode(fetchResponse.error().code()).exception() } else { - Map.empty + java.util.Map.of[TopicPartition, FetchResponseData.PartitionData]() Review Comment: ditto ########## core/src/main/scala/kafka/server/LocalLeaderEndPoint.scala: ########## @@ -226,10 +231,10 @@ class LocalLeaderEndPoint(sourceBroker: BrokerEndPoint, // Set maxWait and minBytes to 0 because the response should return immediately if // the future log has caught up with the current log of the partition val requestBuilder = FetchRequest.Builder.forReplica(version, replicaId, -1, 0, 0, requestMap).setMaxBytes(maxBytes) - Some(ReplicaFetch(requestMap, requestBuilder)) + Optional.of(new ReplicaFetch(requestMap, requestBuilder)) } - ResultWithPartitions(fetchRequestOpt, partitionsWithError) + new ResultWithPartitions[util.Optional[ReplicaFetch]](fetchRequestOpt, partitionsWithError.asJava) Review Comment: ditto ########## core/src/main/scala/kafka/server/RemoteLeaderEndPoint.scala: ########## @@ -199,8 +202,8 @@ class RemoteLeaderEndPoint(logPrefix: String, } val fetchData = builder.build() - val fetchRequestOpt = if (fetchData.sessionPartitions.isEmpty && fetchData.toForget.isEmpty) { - None + val fetchRequestOpt: Optional[ReplicaFetch] = if (fetchData.sessionPartitions.isEmpty && fetchData.toForget.isEmpty) { Review Comment: ditto -- 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