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


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws 
RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = 
fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = 
log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = 
remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, 
epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) 
throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = 
remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == 
FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = 
logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = 
epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? 
Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " 
+ offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not 
exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = 
rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, 
offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is 
greater than or equal to the target offset
+            remoteSegInputStream = 
remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new 
RemoteLogInputStream(remoteSegInputStream);

Review Comment:
   Perf comment: not blocking for this PR
   
   Note that `RemoteLogInputStream` performs a heap allocation here for each 
batch. This allocation will increase GC activity with the size of segment. 
Perhaps, we can use BufferSupplier (we allocate one pool per request handler 
thread, see `requestLocal.bufferSupplier` in KafkaAPI.scala) here in future.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws 
RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = 
fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = 
log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = 
remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, 
epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) 
throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = 
remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == 
FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = 
logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = 
epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? 
Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " 
+ offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not 
exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = 
rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, 
offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is 
greater than or equal to the target offset
+            remoteSegInputStream = 
remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new 
RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, 
offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), 
MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? 
Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when 
there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch 
size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatch.sizeInBytes() > maxBytes) {
+                return new FetchDataInfo(new LogOffsetMetadata(offset), 
MemoryRecords.EMPTY);
+            }
+
+            int updatedFetchSize =
+                    remoteStorageFetchInfo.minOneMessage && 
firstBatch.sizeInBytes() > maxBytes
+                            ? firstBatch.sizeInBytes() : maxBytes;

Review Comment:
   Perf comment: not blocking for this PR
   
   Please calculate `firstBatch.sizeInBytes()` once and reuse. 



##########
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:
   Is this `RejectedExecutionException` propagated to the Consumer fetch? If 
yes, is this a change in the existing interface with the consumer? (please 
correct me if I am wrong but I am not aware of consumer handling or expecting 
`RejectedExecutionException` today.



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -147,6 +164,11 @@ public RemoteLogManager(RemoteLogManagerConfig rlmConfig,
         indexCache = new RemoteIndexCache(1024, remoteLogStorageManager, 
logDir);
         delayInMs = rlmConfig.remoteLogManagerTaskIntervalMs();
         rlmScheduledThreadPool = new 
RLMScheduledThreadPool(rlmConfig.remoteLogManagerThreadPoolSize());
+        remoteStorageReaderThreadPool = new RemoteStorageThreadPool(

Review Comment:
   important
   
   We are not shutting this down in `RemoteLogManager#close()`



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,25 +622,208 @@ public String toString() {
         }
     }
 
-    long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws 
RemoteStorageException {
-        Optional<Long> offset = Optional.empty();
-        Optional<UnifiedLog> maybeLog = 
fetchLog.apply(topicIdPartition.topicPartition());
-        if (maybeLog.isPresent()) {
-            UnifiedLog log = maybeLog.get();
-            Option<LeaderEpochFileCache> maybeLeaderEpochFileCache = 
log.leaderEpochCache();
-            if (maybeLeaderEpochFileCache.isDefined()) {
-                LeaderEpochFileCache cache = maybeLeaderEpochFileCache.get();
-                OptionalInt epoch = cache.latestEpoch();
-                while (!offset.isPresent() && epoch.isPresent()) {
-                    offset = 
remoteLogMetadataManager.highestOffsetForEpoch(topicIdPartition, 
epoch.getAsInt());
-                    epoch = cache.previousEpoch(epoch.getAsInt());
+    public FetchDataInfo read(RemoteStorageFetchInfo remoteStorageFetchInfo) 
throws RemoteStorageException, IOException {
+        int fetchMaxBytes = remoteStorageFetchInfo.fetchMaxBytes;
+        TopicPartition tp = remoteStorageFetchInfo.topicPartition;
+        FetchRequest.PartitionData fetchInfo = 
remoteStorageFetchInfo.fetchInfo;
+
+        boolean includeAbortedTxns = remoteStorageFetchInfo.fetchIsolation == 
FetchIsolation.TXN_COMMITTED;
+
+        long offset = fetchInfo.fetchOffset;
+        int maxBytes = Math.min(fetchMaxBytes, fetchInfo.maxBytes);
+
+        Optional<UnifiedLog> logOptional = fetchLog.apply(tp);
+        OptionalInt epoch = OptionalInt.empty();
+
+        if (logOptional.isPresent()) {
+            Option<LeaderEpochFileCache> leaderEpochCache = 
logOptional.get().leaderEpochCache();
+            if (leaderEpochCache.isDefined()) {
+                epoch = leaderEpochCache.get().epochForOffset(offset);
+            }
+        }
+
+        Optional<RemoteLogSegmentMetadata> rlsMetadataOptional = 
epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadataOptional.isPresent()) {
+            String epochStr = (epoch.isPresent()) ? 
Integer.toString(epoch.getAsInt()) : "NOT AVAILABLE";
+            throw new OffsetOutOfRangeException("Received request for offset " 
+ offset + " for leader epoch "
+                    + epochStr + " and partition " + tp + " which does not 
exist in remote tier.");
+        }
+
+        RemoteLogSegmentMetadata remoteLogSegmentMetadata = 
rlsMetadataOptional.get();
+        int startPos = lookupPositionForOffset(remoteLogSegmentMetadata, 
offset);
+        InputStream remoteSegInputStream = null;
+        try {
+            // Search forward for the position of the last offset that is 
greater than or equal to the target offset
+            remoteSegInputStream = 
remoteLogStorageManager.fetchLogSegment(remoteLogSegmentMetadata, startPos);
+            RemoteLogInputStream remoteLogInputStream = new 
RemoteLogInputStream(remoteSegInputStream);
+
+            RecordBatch firstBatch = findFirstBatch(remoteLogInputStream, 
offset);
+
+            if (firstBatch == null)
+                return new FetchDataInfo(new LogOffsetMetadata(offset), 
MemoryRecords.EMPTY, false,
+                        includeAbortedTxns ? 
Optional.of(Collections.emptyList()) : Optional.empty());
+
+            // An empty record is sent instead of an incomplete batch when 
there is no minimum-one-message constraint
+            // and for FetchRequest version 3 and above and the first batch 
size is more than maximum bytes that can be sent.
+            if (!remoteStorageFetchInfo.minOneMessage &&
+                    !remoteStorageFetchInfo.hardMaxBytesLimit &&
+                    firstBatch.sizeInBytes() > maxBytes) {

Review Comment:
   This and the above `firstBatch == null` condition can be combined since they 
produce the same result.



##########
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 = {
+    // cancel the remote storage read task, if it has not been executed yet
+    remoteFetchTask.cancel(false)

Review Comment:
   1. Why aren't we interrupting this with a true argument?
   2. We should probably check the return and if we are unable to cancel, we 
should log a debug statement here.



##########
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:
   non blocking comment
   
   we probably need metrics on expiration of delated remote fetch here. 



##########
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 = {
+    // cancel the remote storage read task, if it has not been executed yet
+    remoteFetchTask.cancel(false)
+  }
+
+  /**
+   * Upon completion, read whatever data is available and pass to the complete 
callback
+   */
+  override def onComplete():Unit = {
+    val fetchPartitionData = localReadResults.map { case (tp, result) =>
+      if (tp.topicPartition().equals(remoteFetchInfo.topicPartition) && 
remoteFetchResult.isDone
+        && result.exception.isEmpty && 
result.info.delayedRemoteStorageFetch.isPresent) {

Review Comment:
   nit
   
   s/result.Exception.isEmpty/(result.error == Errors.NONE)
   
   Asking because it makes the code more clear since we are using result.error 
in the else to populate the reponse.



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