kamalcph commented on code in PR #16602:
URL: https://github.com/apache/kafka/pull/16602#discussion_r1759290805


##########
core/src/main/scala/kafka/server/DelayedRemoteListOffsets.scala:
##########
@@ -0,0 +1,174 @@
+/**
+ * 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 kafka.server
+
+import com.yammer.metrics.core.Meter
+import kafka.log.AsyncOffsetReadFutureHolder
+import kafka.utils.Implicits._
+import kafka.utils.Pool
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.errors.ApiException
+import 
org.apache.kafka.common.message.ListOffsetsResponseData.{ListOffsetsPartitionResponse,
 ListOffsetsTopicResponse}
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.FileRecords.TimestampAndOffset
+import org.apache.kafka.common.requests.ListOffsetsResponse
+import org.apache.kafka.server.metrics.KafkaMetricsGroup
+
+import java.util.concurrent.TimeUnit
+import scala.collection.{Map, mutable}
+import scala.jdk.CollectionConverters._
+
+case class ListOffsetsPartitionStatus(var responseOpt: 
Option[ListOffsetsPartitionResponse] = None,
+                                      futureHolderOpt: 
Option[AsyncOffsetReadFutureHolder[Either[Exception, 
Option[TimestampAndOffset]]]] = None,
+                                      lastFetchableOffset: Option[Long] = None,
+                                      maybeOffsetsError: Option[ApiException] 
= None) {
+  @volatile var completed = false
+
+  override def toString: String = {
+    s"[responseOpt: $responseOpt, lastFetchableOffset: $lastFetchableOffset, " 
+
+      s"maybeOffsetsError: $maybeOffsetsError, completed: $completed]"
+  }
+}
+
+case class ListOffsetsMetadata(statusByPartition: mutable.Map[TopicPartition, 
ListOffsetsPartitionStatus]) {
+
+  override def toString: String = {
+    s"ListOffsetsMetadata(statusByPartition=$statusByPartition)"
+  }
+}
+
+class DelayedRemoteListOffsets(delayMs: Long,
+                               version: Int,
+                               metadata: ListOffsetsMetadata,
+                               responseCallback: 
List[ListOffsetsTopicResponse] => Unit) extends DelayedOperation(delayMs) {
+
+  // Mark the status as completed, if there is no async task to track.
+  // If there is a task to track, then build the response as REQUEST_TIMED_OUT 
by default.
+  metadata.statusByPartition.forKeyValue { (topicPartition, status) =>
+    status.completed = status.futureHolderOpt.isEmpty
+    if (status.futureHolderOpt.isDefined) {
+      status.responseOpt = Some(buildErrorResponse(Errors.REQUEST_TIMED_OUT, 
topicPartition.partition()))

Review Comment:
   yes, that's true. Assume that one LIST_OFFSETS request wants to query the 
`offsetForTimestamp` for 10 partitions, those 10 partitions are handled in 
concurrent fashion, provided that the remote-log-reader threads are available. 
If any one thread completes, then it marks that partition status as completed 
and checks for the statuses of all the other partitions. 
   
   The variable `completed` is accessed by multiple remote-log-reader threads 
so marked it as volatile and used it for computation instead of `responseOpt`.



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