You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "showuon (via GitHub)" <gi...@apache.org> on 2023/04/12 10:43:37 UTC

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

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


##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.log.remote;
+
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading remote bytes for {}", fetchInfo.topicPartition);
+
+            FetchDataInfo fetchDataInfo = rlm.read(fetchInfo);
+            result = new RemoteLogReadResult(Optional.of(fetchDataInfo), Optional.empty());
+        } catch (OffsetOutOfRangeException e) {
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        } catch (Exception e) {
+            logger.error("Error occurred while reading the remote data for {}", fetchInfo.topicPartition, e);
+            result = new RemoteLogReadResult(Optional.empty(), Optional.of(e));
+        }
+
+        logger.debug("Finished reading remote bytes for {}", fetchInfo.topicPartition);

Review Comment:
   ditto



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -619,6 +811,49 @@ long findHighestRemoteOffset(TopicIdPartition topicIdPartition) throws RemoteSto
         return offset.orElse(-1L);
     }
 
+    /**
+     * A remote log read task returned by asyncRead(). The caller of asyncRead() can use this object to cancel a
+     * pending task or check if the task is done.
+     */
+    public class AsyncReadTask {

Review Comment:
   I don't see `AsyncReadTask` is used anywhere in the code. Could we remove it?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1196,17 +1252,45 @@ class ReplicaManager(val config: KafkaConfig,
                  _: FencedLeaderEpochException |
                  _: ReplicaNotAvailableException |
                  _: KafkaStorageException |
-                 _: OffsetOutOfRangeException |
                  _: InconsistentTopicIdException) =>
-          LogReadResult(info = new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY),
-            divergingEpoch = None,
-            highWatermark = UnifiedLog.UnknownOffset,
-            leaderLogStartOffset = UnifiedLog.UnknownOffset,
-            leaderLogEndOffset = UnifiedLog.UnknownOffset,
-            followerLogStartOffset = UnifiedLog.UnknownOffset,
-            fetchTimeMs = -1L,
-            lastStableOffset = None,
-            exception = Some(e))
+          createLogReadResult(e)
+        case e: OffsetOutOfRangeException =>
+          // In case of offset out of range errors, check for remote log manager for non-compacted topics
+          // to fetch from remote storage. `log` instance should not be null here as that would have caught earlier with
+          // NotLeaderForPartitionException or ReplicaNotAvailableException.
+          // If it is from a follower then send the offset metadata but not the records data as that can be fetched

Review Comment:
   nit: `as [the follower] can be fetched from the remote store directly`



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -1083,48 +1095,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)
+                else
+                  result
+              }
+
+              tp -> r.toFetchPartitionData(false)
+            }
+            responseCallback(fetchPartitionData)
+            return
+        }
+
+        // If there is remote data, we will read remote data, instead of waiting for new data.

Review Comment:
   I think this comment should be put on L1100 after `if (remoteFetchInfo.isPresent) {`



##########
core/src/main/java/kafka/log/remote/RemoteLogReader.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.log.remote;
+
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.storage.internals.log.FetchDataInfo;
+import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo;
+import org.slf4j.Logger;
+
+import java.util.Optional;
+import java.util.concurrent.Callable;
+import java.util.function.Consumer;
+
+public class RemoteLogReader implements Callable<Void> {
+
+    private final Logger logger;
+    private final RemoteStorageFetchInfo fetchInfo;
+    private final RemoteLogManager rlm;
+    private final Consumer<RemoteLogReadResult> callback;
+
+    public RemoteLogReader(RemoteStorageFetchInfo fetchInfo,
+                           RemoteLogManager rlm,
+                           Consumer<RemoteLogReadResult> callback) {
+        this.fetchInfo = fetchInfo;
+        this.rlm = rlm;
+        this.callback = callback;
+        logger = new LogContext() {
+            @Override
+            public String logPrefix() {
+                return "[" + Thread.currentThread().getName() + "]";
+            }
+        }.logger(RemoteLogReader.class);
+    }
+
+    @Override
+    public Void call() {
+        RemoteLogReadResult result;
+        try {
+            logger.debug("Reading remote bytes for {}", fetchInfo.topicPartition);

Review Comment:
   The `remote bytes` is confusing me. How about `Reading [records] from remote storage for {}`?



##########
core/src/main/scala/kafka/server/DelayedRemoteFetch.scala:
##########
@@ -0,0 +1,121 @@
+/**
+ * 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 kafka.log.remote.RemoteLogReadResult
+import org.apache.kafka.common.TopicIdPartition
+import org.apache.kafka.common.errors._
+import org.apache.kafka.storage.internals.log.{FetchParams, FetchPartitionData, LogOffsetMetadata, 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,
+                         quota: ReplicaQuota,
+                         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
+   * Case e: This broker is the leader, but the requested epoch is now fenced
+   *
+   * 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
+        val fetchLeaderEpoch = fetchStatus.fetchInfo.currentLeaderEpoch
+        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 _: FencedLeaderEpochException => // Case e
+            debug(s"Broker is the leader of partition $topicPartition, but the requested epoch " +
+              s"$fetchLeaderEpoch is fenced by the latest leader epoch, satisfy $fetchParams immediately")
+            return forceComplete()
+          case _: NotLeaderOrFollowerException =>  // Case a
+            debug("Broker is no longer the leader of %s, satisfy %s immediately".format(topicPartition, fetchParams))
+            return forceComplete()
+        }
+    }
+    if (remoteFetchResult.isDone)

Review Comment:
   should we add `case C` for this case?



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -600,6 +622,176 @@ public String toString() {
         }
     }
 
+    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> rlsMetadata = epoch.isPresent()
+                ? fetchRemoteLogSegmentMetadata(tp, epoch.getAsInt(), offset)
+                : Optional.empty();
+
+        if (!rlsMetadata.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. Try again later.");

Review Comment:
   Are we sure `Try again later` can fix this issue? Especially, it's from application, this log doesn't help at all. Could we remove it? Thoughts?



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