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


##########
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:
   1. The expiration metrics are being emitted by the individual purgatory. 
(eg) 
[DelayedRemoteListOffsetsMetrics#recordExpiration](https://sourcegraph.com/github.com/apache/kafka/-/blob/core/src/main/scala/kafka/server/DelayedRemoteListOffsets.scala?L76)
   2. Agree, this will improve the performance. Will take the custom 
DelayedFuturePurgatory changes separately as it is a big change. 



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