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


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -1263,7 +1263,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
    *         None if no such message is found.
    */
   @nowarn("cat=deprecation")
-  def fetchOffsetByTimestamp(targetTimestamp: Long, remoteLogManager: 
Option[RemoteLogManager] = None): Option[TimestampAndOffset] = {
+  def fetchOffsetByTimestamp(targetTimestamp: Long, remoteLogManager: 
Option[RemoteLogManager] = None): OffsetResultHolder = {

Review Comment:
   Could we change the description of the return value accordingly?



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

Review Comment:
   `ReplicaManager` calls `completeDelayedFetchOrProduceRequests` when a 
replica is removed from the broker or becomes a follower to wake up pending 
produce/fetch request early. Should we do the same for pending remoteListOffset 
requests?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1447,11 +1466,123 @@ 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), _) =>
+                // This case is for normal topic that does not have remote 
storage.
+                var partitionResponse = buildErrorResponse(Errors.NONE, 
partition)
+                if (resultHolder.lastFetchableOffset.isDefined &&
+                  found.offset >= resultHolder.lastFetchableOffset.get) {
+                  resultHolder.maybeOffsetsError.map(e => throw e)
+                } else {
+                  partitionResponse = new ListOffsetsPartitionResponse()
+                    .setPartitionIndex(partition.partitionIndex)
+                    .setErrorCode(Errors.NONE.code)
+                    .setTimestamp(found.timestamp)
+                    .setOffset(found.offset)
+                  if (found.leaderEpoch.isPresent && version >= 4)
+                    partitionResponse.setLeaderEpoch(found.leaderEpoch.get)
+                }
+                ListOffsetsPartitionStatus(Some(partitionResponse))
+              case OffsetResultHolder(None, None) =>
+                // This is an empty offset response scenario
+                resultHolder.maybeOffsetsError.map(e => throw e)
+                
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.NONE, partition)))
+              case OffsetResultHolder(None, Some(futureHolder)) =>
+                // This case is for topic enabled with remote storage and we 
want to search the timestamp in
+                // remote storage using async fashion.
+                ListOffsetsPartitionStatus(None, Some(futureHolder), 
resultHolder.lastFetchableOffset, resultHolder.maybeOffsetsError)
+            }
+            statusByPartition += topicPartition -> status
+          } catch {
+            // NOTE: These exceptions are special cases since these error 
messages are typically transient or the client
+            // would have received a clear exception and there is no value in 
logging the entire stack trace for the same
+            case e @ (_ : UnknownTopicOrPartitionException |
+                      _ : NotLeaderOrFollowerException |
+                      _ : UnknownLeaderEpochException |
+                      _ : FencedLeaderEpochException |
+                      _ : KafkaStorageException |
+                      _ : UnsupportedForMessageFormatException) =>
+              debug(s"Offset request with correlation id $correlationId from 
client $clientId on " +
+                s"partition $topicPartition failed due to ${e.getMessage}")
+              statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+
+            // Only V5 and newer ListOffset calls should get 
OFFSET_NOT_AVAILABLE
+            case e: OffsetNotAvailableException =>
+              if (version >= 5) {
+                statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+              } else {
+                statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.LEADER_NOT_AVAILABLE, 
partition)))
+              }
+
+            case e: Throwable =>
+              error("Error while responding to offset request", e)
+              statusByPartition += topicPartition -> 
ListOffsetsPartitionStatus(Some(buildErrorResponse(Errors.forException(e), 
partition)))
+          }
+        }
+      }
+    }
+
+    if (delayedRemoteListOffsetsRequired(statusByPartition)) {
+      val timeout = 
config.remoteLogManagerConfig.remoteListOffsetsRequestTimeoutMs()
+      // create delayed remote list offsets operation
+      val delayedRemoteListOffsets = new DelayedRemoteListOffsets(timeout, 
version, ListOffsetsMetadata(statusByPartition), responseCallback)
+      // create a list of (topic, partition) pairs to use as keys for this 
delayed remote list offsets operation
+      val listOffsetsRequestKeys = 
statusByPartition.keys.map(TopicPartitionOperationKey(_)).toSeq
+      // try to complete the request immediately, otherwise put it into the 
purgatory
+      
delayedRemoteListOffsetsPurgatory.tryCompleteElseWatch(delayedRemoteListOffsets,
 listOffsetsRequestKeys)

Review Comment:
   `listOffsetsRequestKeys` is a bit weird. It is based on a topicPartition and 
the purgatory is checked on that key every time a remote listOffset task 
completes. However, the completion of such a task has no impact on other 
pending listOffset requests on the same partition.
   
   The only reason we need the purgatory is really just for the expiration 
logic after the timeout if we chain all the futures together. Perhaps, using 
the pattern of DelayedFuturePurgatory is more intuitive?



##########
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]) {

Review Comment:
   Hmm, why do we need this wrapper class? Could we just use the `Map` directly?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -263,6 +270,10 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
         );
     }
 
+    public void 
setDelayedOperationPurgatory(DelayedOperationPurgatory<DelayedRemoteListOffsets>
 delayedRemoteListOffsetsPurgatory) {
+        this.delayedRemoteListOffsetsPurgatory = 
delayedRemoteListOffsetsPurgatory;

Review Comment:
   delayedRemoteListOffsetsPurgatory is written and read by different threads. 
Does it need to be volatile?



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

Review Comment:
   `responseOpt` can be written and read by different threads. Should it be 
volatile?



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