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


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -620,6 +631,23 @@ private Optional<Integer> maybeLeaderEpoch(int 
leaderEpoch) {
         return leaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH ? 
Optional.empty() : Optional.of(leaderEpoch);
     }
 
+    public AsyncOffsetReadFutureHolder<Either<Exception, 
Option<FileRecords.TimestampAndOffset>>> asyncOffsetRead(
+            TopicPartition topicPartition,
+            Long timestamp,
+            Long startingOffset,
+            LeaderEpochFileCache leaderEpochCache,
+            Supplier<Option<FileRecords.TimestampAndOffset>> searchLocalLog) {
+        CompletableFuture<Either<Exception, 
Option<FileRecords.TimestampAndOffset>>> taskFuture = new CompletableFuture<>();
+        Future<Void> jobFuture = remoteStorageReaderThreadPool.submit(
+                new RemoteLogOffsetReader(this, topicPartition, timestamp, 
startingOffset, leaderEpochCache, searchLocalLog, result -> {
+                    TopicPartitionOperationKey key = new 
TopicPartitionOperationKey(topicPartition.topic(), topicPartition.partition());
+                    taskFuture.complete(result);
+                    delayedRemoteListOffsetsPurgatory.checkAndComplete(key);
+                })
+        );
+        return new AsyncOffsetReadFutureHolder<>(jobFuture, taskFuture);

Review Comment:
   Pardon me, why we need two futures here? Is `CompletableFuture.supplyAsync` 
unsuitable to this case? for example:
   ```java
           CompletableFuture<Optional<FileRecords.TimestampAndOffset>> 
taskFuture = CompletableFuture.supplyAsync(() -> {
               try {
                   // If it is not found in remote storage, then search in the 
local storage starting with local log start offset.
                   Optional<FileRecords.TimestampAndOffset> rval = 
findOffsetByTimestamp(topicPartition, timestamp, startingOffset, 
leaderEpochCache);
                   if (rval.isPresent()) return rval;
                   return OptionConverters.toJava(searchLocalLog.get());
               } catch (Exception e) {
                   // NOTE: All the exceptions from the secondary storage are 
catched instead of only the KafkaException.
                   LOGGER.error("Error occurred while reading the remote log 
offset for {}", topicPartition, e);
                   throw new RuntimeException(e);
               }
           }, remoteStorageReaderThreadPool);
   ```
   



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1447,11 +1466,119 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
+  def fetchOffset(topics: Seq[ListOffsetsTopic],
+                  duplicatePartitions: Set[TopicPartition],
+                  isolationLevel: IsolationLevel,
+                  replicaId: Int,
+                  clientId: String,
+                  correlationId: Int,
+                  version: Short,
+                  buildErrorResponse: (Errors, ListOffsetsPartition) => 
ListOffsetsPartitionResponse,
+                  responseCallback: List[ListOffsetsTopicResponse] => Unit): 
Unit = {
+    val statusByPartition = mutable.Map[TopicPartition, 
ListOffsetsPartitionStatus]()
+    topics.foreach { topic =>
+      topic.partitions.asScala.foreach { partition =>
+        val topicPartition = new TopicPartition(topic.name, 
partition.partitionIndex)
+        if (duplicatePartitions.contains(topicPartition)) {
+          debug(s"OffsetRequest with correlation id $correlationId from client 
$clientId on partition $topicPartition " +
+            s"failed because the partition is duplicated in the request.")
+          statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.INVALID_REQUEST, 
partition)))
+        } else if (isListOffsetsTimestampUnsupported(partition.timestamp(), 
version)) {
+          statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.UNSUPPORTED_VERSION, 
partition)))
+        } else {
+          try {
+            val fetchOnlyFromLeader = replicaId != 
ListOffsetsRequest.DEBUGGING_REPLICA_ID
+            val isClientRequest = replicaId == 
ListOffsetsRequest.CONSUMER_REPLICA_ID
+            val isolationLevelOpt = if (isClientRequest)
+              Some(isolationLevel)
+            else
+              None
+
+            val resultHolder = fetchOffsetForTimestamp(topicPartition,
+              partition.timestamp,
+              isolationLevelOpt,
+              if (partition.currentLeaderEpoch == 
ListOffsetsResponse.UNKNOWN_EPOCH) Optional.empty() else 
Optional.of(partition.currentLeaderEpoch),
+              fetchOnlyFromLeader)
+
+            val status = resultHolder match {
+              case OffsetResultHolder(Some(found), _) =>

Review Comment:
   The data structure gets complicated in this path. If those new structures 
serve for "remote" only, could you please consider defining a subclass of 
`TimestampAndOffset` to have data used by remote only?
   
   or please add comments for those cases at least?



##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1336,34 +1336,39 @@ class UnifiedLog(@volatile var logStartOffset: Long,
         val maxTimestampSoFar = 
latestTimestampSegment.readMaxTimestampAndOffsetSoFar
         // lookup the position of batch to avoid extra I/O
         val position = 
latestTimestampSegment.offsetIndex.lookup(maxTimestampSoFar.offset)
-        latestTimestampSegment.log.batchesFrom(position.position).asScala
+        val timestampAndOffsetOpt = 
latestTimestampSegment.log.batchesFrom(position.position).asScala
           .find(_.maxTimestamp() == maxTimestampSoFar.timestamp)
           .flatMap(batch => batch.offsetOfMaxTimestamp().asScala.map(new 
TimestampAndOffset(batch.maxTimestamp(), _,
             Optional.of[Integer](batch.partitionLeaderEpoch()).filter(_ >= 
0))))
+        OffsetResultHolder(timestampAndOffsetOpt)

Review Comment:
   Pardon me, why `MAX_TIMESTAMP` does not consider the records in the remote 
storage?



##########
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:
   Pardon me, why setting `responseOpt` early? If we keep the `responseOpt` be 
None, we can reuse `responseOpt` to evaluate "completed"



##########
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()))
+    }
+    trace(s"Initial partition status for $topicPartition is $status")
+  }
+
+  /**
+   * Call-back to execute when a delayed operation gets expired and hence 
forced to complete.
+   */
+  override def onExpiration(): Unit = {
+    metadata.statusByPartition.forKeyValue { (topicPartition, status) =>
+      if (!status.completed) {
+        debug(s"Expiring list offset request for partition $topicPartition 
with status $status")
+        status.futureHolderOpt.foreach(futureHolder => 
futureHolder.jobFuture.cancel(true))
+        DelayedRemoteListOffsetsMetrics.recordExpiration(topicPartition)
+      }
+    }
+  }
+
+  /**
+   * Process for completing an operation; This function needs to be defined
+   * in subclasses and will be called exactly once in forceComplete()
+   */
+  override def onComplete(): Unit = {
+    val responseTopics = metadata.statusByPartition.groupBy(e => 
e._1.topic()).map {
+      case (topic, status) =>
+        new 
ListOffsetsTopicResponse().setName(topic).setPartitions(status.values.flatMap(s 
=> s.responseOpt).toList.asJava)
+    }.toList
+    responseCallback(responseTopics)
+  }
+
+  /**
+   * Try to complete the delayed operation by first checking if the operation
+   * can be completed by now. If yes execute the completion logic by calling
+   * forceComplete() and return true iff forceComplete returns true; otherwise 
return false
+   *
+   * This function needs to be defined in subclasses
+   */
+  override def tryComplete(): Boolean = {
+    var completable = true
+    metadata.statusByPartition.forKeyValue { (partition, status) =>
+      if (!status.completed) {

Review Comment:
   Could we check the status of futures instead? for example:
   ```
     def completable = status.futureHolderOpt.isEmpty || 
status.futureHolderOpt.get.jobFuture.isDone
   ```



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