You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "satishd (via GitHub)" <gi...@apache.org> on 2023/05/01 11:17:06 UTC

[GitHub] [kafka] satishd commented on a diff in pull request #13535: KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory.

satishd commented on code in PR #13535:
URL: https://github.com/apache/kafka/pull/13535#discussion_r1181518576


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1160,48 +1171,100 @@ class ReplicaManager(val config: KafkaConfig,
           fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData))
         })
       }
-      val delayedFetch = new DelayedFetch(
-        params = params,
-        fetchPartitionStatus = fetchPartitionStatus,
-        replicaManager = this,
-        quota = quota,
-        responseCallback = responseCallback
-      )
-
-      // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
-      val delayedFetchKeys = fetchPartitionStatus.map { case (tp, _) => TopicPartitionOperationKey(tp) }
-
-      // try to complete the request immediately, otherwise put it into the purgatory;
-      // this is because while the delayed fetch operation is being created, new requests
-      // may arrive and hence make this operation completable.
-      delayedFetchPurgatory.tryCompleteElseWatch(delayedFetch, delayedFetchKeys)
+
+      if (remoteFetchInfo.isPresent) {
+        val key = new TopicPartitionOperationKey(remoteFetchInfo.get.topicPartition.topic(), remoteFetchInfo.get.topicPartition.partition())
+        val remoteFetchResult = new CompletableFuture[RemoteLogReadResult]
+        var remoteFetchTask: Future[Void] = null
+        try {
+          remoteFetchTask = remoteLogManager.get.asyncRead(remoteFetchInfo.get, (result: RemoteLogReadResult) => {
+            remoteFetchResult.complete(result)
+            delayedRemoteFetchPurgatory.checkAndComplete(key)
+          })
+        } catch {
+          // if the task queue of remote storage reader thread pool is full, return what we currently have
+          // (the data read from local log segment for the other topic-partitions) and an error for the topic-partition that
+          // we couldn't read from remote storage
+          case e: RejectedExecutionException =>
+            val fetchPartitionData = logReadResults.map { case (tp, result) =>
+              val r = {
+                if (tp.topicPartition().equals(remoteFetchInfo.get.topicPartition))
+                  createLogReadResult(e)

Review Comment:
   This error is [propagated](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ReplicaManager.scala#L86) as unexpected error (UnknownServerException) to the consumer client and it is already handled. 



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,112 @@
+/**
+ * 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 org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo}
+
+import java.util.concurrent.{CompletableFuture, Future}
+import java.util.{Optional, OptionalInt, OptionalLong}
+import scala.collection._
+
+/**
+ * A remote fetch operation that can be created by the replica manager and watched
+ * in the remote fetch operation purgatory
+ */
+class DelayedRemoteFetch(remoteFetchTask: Future[Void],
+                         remoteFetchResult: CompletableFuture[RemoteLogReadResult],
+                         remoteFetchInfo: RemoteStorageFetchInfo,
+                         fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)],
+                         fetchParams: FetchParams,
+                         localReadResults: Seq[(TopicIdPartition, LogReadResult)],
+                         replicaManager: ReplicaManager,
+                         responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit)
+  extends DelayedOperation(fetchParams.maxWaitMs) {
+
+  /**
+   * The operation can be completed if:
+   *
+   * Case a: This broker is no longer the leader of the partition it tries to fetch
+   * Case b: This broker does not know the partition it tries to fetch
+   * Case c: The remote storage read request completed (succeeded or failed)
+   * Case d: The partition is in an offline log directory on this broker
+   *
+   * Upon completion, should return whatever data is available for each valid partition
+   */
+  override def tryComplete(): Boolean = {
+    fetchPartitionStatus.foreach {
+      case (topicPartition, fetchStatus) =>
+        val fetchOffset = fetchStatus.startOffsetMetadata
+        try {
+          if (fetchOffset != LogOffsetMetadata.UNKNOWN_OFFSET_METADATA) {
+            replicaManager.getPartitionOrException(topicPartition.topicPartition())
+          }
+        } catch {
+          case _: KafkaStorageException => // Case d
+            debug(s"Partition $topicPartition is in an offline log directory, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: UnknownTopicOrPartitionException => // Case b
+            debug(s"Broker no longer knows of partition $topicPartition, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader or follower of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone) // Case c
+      forceComplete()
+    else
+      false
+  }
+
+  override def onExpiration():Unit = {

Review Comment:
   Yes, we have in the internal branch. There are other metrics too related to tiered storage.  They will be added in a followup PR. 
   
   <img width="1415" alt="image" src="https://user-images.githubusercontent.com/2577761/235444150-338d3472-c511-4252-bc56-1afa31c1ad2b.png">
   
   



-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org