From 7ab02f4b052e56e2f6efc857b6438da4fb03455c Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Mon, 2 Jun 2025 13:00:14 +0800 Subject: [PATCH 01/17] KAFKA-19340: Move DelayedRemoteFetch to the storage module --- .../kafka/server/share/DelayedShareFetch.java | 2 +- .../scala/kafka/server/DelayedFetch.scala | 11 +- .../kafka/server/DelayedRemoteFetch.scala | 135 ------------- .../scala/kafka/server/ReplicaManager.scala | 24 +-- .../kafka/server/FetchPartitionStatus.java | 39 ++++ .../apache/kafka/server/LogReadResult.java | 15 ++ .../server/purgatory/DelayedRemoteFetch.java | 188 ++++++++++++++++++ 7 files changed, 250 insertions(+), 164 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/DelayedRemoteFetch.scala create mode 100644 server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java create mode 100644 server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index fce50613becf8..4e1fa3c1ae8da 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -832,7 +832,7 @@ private void completeRemoteStorageShareFetchRequest() { new ShareFetchPartitionData( remoteFetch.topicIdPartition(), partitionsAcquired.get(remoteFetch.topicIdPartition()), - ReplicaManager.createLogReadResult(error).toFetchPartitionData(false) + new LogReadResult(error).toFetchPartitionData(false) ) ); } else { diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 74a3e2b1a2997..f0d0cb22fc904 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -24,8 +24,8 @@ import java.util.concurrent.TimeUnit import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.errors._ import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} +import org.apache.kafka.server.FetchPartitionStatus import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.purgatory.DelayedOperation import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} @@ -34,15 +34,6 @@ import org.apache.kafka.storage.internals.log.LogOffsetMetadata import scala.collection._ import scala.jdk.CollectionConverters._ -case class FetchPartitionStatus(startOffsetMetadata: LogOffsetMetadata, fetchInfo: PartitionData) { - - override def toString: String = { - "[startOffsetMetadata: " + startOffsetMetadata + - ", fetchInfo: " + fetchInfo + - "]" - } -} - /** * A delayed fetch operation that can be created by the replica manager and watched * in the fetch operation purgatory diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala deleted file mode 100644 index 317c8dd4ac9e6..0000000000000 --- a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala +++ /dev/null @@ -1,135 +0,0 @@ -/** - * 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.utils.Logging -import org.apache.kafka.common.TopicIdPartition -import org.apache.kafka.common.errors._ -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.server.LogReadResult -import org.apache.kafka.server.metrics.KafkaMetricsGroup -import org.apache.kafka.server.purgatory.DelayedOperation -import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData} -import org.apache.kafka.storage.internals.log.{LogOffsetMetadata, RemoteLogReadResult, RemoteStorageFetchInfo} - -import java.util.concurrent.{CompletableFuture, Future, TimeUnit} -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, - remoteFetchMaxWaitMs: Long, - fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)], - fetchParams: FetchParams, - localReadResults: Seq[(TopicIdPartition, LogReadResult)], - replicaManager: ReplicaManager, - responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit) - extends DelayedOperation(remoteFetchMaxWaitMs) with Logging { - - if (fetchParams.isFromFollower) { - throw new IllegalStateException(s"The follower should not invoke remote fetch. Fetch params are: $fetchParams") - } - - /** - * 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 and - // avoid interrupting the task if it is already running as it may force closing opened/cached resources as transaction index. - val cancelled = remoteFetchTask.cancel(false) - if (!cancelled) debug(s"Remote fetch task for RemoteStorageFetchInfo: $remoteFetchInfo could not be cancelled and its isDone value is ${remoteFetchTask.isDone}") - - DelayedRemoteFetchMetrics.expiredRequestMeter.mark() - } - - /** - * 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.error == Errors.NONE - && result.info.delayedRemoteStorageFetch.isPresent) { - if (remoteFetchResult.get.error.isPresent) { - tp -> ReplicaManager.createLogReadResult(remoteFetchResult.get.error.get).toFetchPartitionData(false) - } else { - val info = remoteFetchResult.get.fetchDataInfo.get - tp -> new FetchPartitionData( - result.error, - result.highWatermark, - result.leaderLogStartOffset, - info.records, - Optional.empty(), - if (result.lastStableOffset.isPresent) OptionalLong.of(result.lastStableOffset.getAsLong) else OptionalLong.empty(), - info.abortedTransactions, - if (result.preferredReadReplica.isPresent) OptionalInt.of(result.preferredReadReplica.getAsInt) else OptionalInt.empty(), - false) - } - } else { - tp -> result.toFetchPartitionData(false) - } - } - - responseCallback(fetchPartitionData) - } -} - -object DelayedRemoteFetchMetrics { - private val metricsGroup = new KafkaMetricsGroup(DelayedRemoteFetchMetrics.getClass) - val expiredRequestMeter: Meter = metricsGroup.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS) -} diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 32f51acf77fda..8bf7b13e78e90 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -56,12 +56,12 @@ import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.log.remote.storage.RemoteLogManager import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.network.BrokerEndPoint -import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey} +import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteFetch, DelayedRemoteListOffsets, DeleteRecordsPartitionStatus, ListOffsetsPartitionStatus, TopicPartitionOperationKey} import org.apache.kafka.server.share.fetch.{DelayedShareFetchKey, DelayedShareFetchPartitionKey} import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData} import org.apache.kafka.server.util.timer.{SystemTimer, TimerTask} import org.apache.kafka.server.util.{Scheduler, ShutdownableThread} -import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, LogReadResult, common} +import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, FetchPartitionStatus, LogReadResult, common} import org.apache.kafka.storage.internals.checkpoint.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogReadInfo, OffsetResultHolder, RecordValidationException, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -188,18 +188,6 @@ object ReplicaManager { Optional.of(e)) } - def createLogReadResult(e: Throwable): LogReadResult = { - new LogReadResult(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), - Optional.empty(), - UnifiedLog.UNKNOWN_OFFSET, - UnifiedLog.UNKNOWN_OFFSET, - UnifiedLog.UNKNOWN_OFFSET, - UnifiedLog.UNKNOWN_OFFSET, - -1L, - OptionalLong.empty(), - Optional.of(e)) - } - private[server] def isListOffsetsTimestampUnsupported(timestamp: JLong, version: Short): Boolean = { timestamp < 0 && (!timestampMinSupportedVersion.contains(timestamp) || version < timestampMinSupportedVersion(timestamp)) @@ -1600,7 +1588,7 @@ class ReplicaManager(val config: KafkaConfig, case e: RejectedExecutionException => // Return the error if any in scheduling the remote fetch task warn("Unable to fetch data from remote storage", e) - return Some(createLogReadResult(e)) + return Some(new LogReadResult(e)) } val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong @@ -1683,7 +1671,7 @@ class ReplicaManager(val config: KafkaConfig, fetchInfos.foreach { case (topicIdPartition, partitionData) => logReadResultMap.get(topicIdPartition).foreach(logReadResult => { val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata - fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus(logOffsetMetadata, partitionData)) + fetchPartitionStatus += (topicIdPartition -> new FetchPartitionStatus(logOffsetMetadata, partitionData)) }) } @@ -1820,7 +1808,7 @@ class ReplicaManager(val config: KafkaConfig, _: ReplicaNotAvailableException | _: KafkaStorageException | _: InconsistentTopicIdException) => - createLogReadResult(e) + new LogReadResult(e) case e: OffsetOutOfRangeException => handleOffsetOutOfRangeError(tp, params, fetchInfo, adjustedMaxBytes, minOneMessage, log, fetchTimeMs, e) case e: Throwable => @@ -1916,7 +1904,7 @@ class ReplicaManager(val config: KafkaConfig, Optional.empty[Throwable]()) } } else { - createLogReadResult(exception) + new LogReadResult(exception) } } diff --git a/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java b/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java new file mode 100644 index 0000000000000..06dcfa7b840ea --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java @@ -0,0 +1,39 @@ +/* + * 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 org.apache.kafka.server; + +import org.apache.kafka.common.requests.FetchRequest.PartitionData; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; + +/** + * A class containing log offset metadata and fetch info for a topic partition. + */ +public record FetchPartitionStatus( + LogOffsetMetadata startOffsetMetadata, + PartitionData fetchInfo +) { + + @Override + public String toString() { + return String.format( + "[startOffsetMetadata: %s, fetchInfo: %s", + startOffsetMetadata, + fetchInfo + ); + } +} diff --git a/server/src/main/java/org/apache/kafka/server/LogReadResult.java b/server/src/main/java/org/apache/kafka/server/LogReadResult.java index 203654391af29..9a39eca8e202b 100644 --- a/server/src/main/java/org/apache/kafka/server/LogReadResult.java +++ b/server/src/main/java/org/apache/kafka/server/LogReadResult.java @@ -18,8 +18,11 @@ import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.server.storage.log.FetchPartitionData; import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.UnifiedLog; import java.util.Optional; import java.util.OptionalInt; @@ -93,6 +96,18 @@ public LogReadResult( fetchTimeMs, lastStableOffset, preferredReadReplica, Optional.empty()); } + public LogReadResult(Throwable e) { + this(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), + Optional.empty(), + UnifiedLog.UNKNOWN_OFFSET, + UnifiedLog.UNKNOWN_OFFSET, + UnifiedLog.UNKNOWN_OFFSET, + UnifiedLog.UNKNOWN_OFFSET, + -1L, + OptionalLong.empty(), + Optional.of(e)); + } + public Errors error() { if (exception.isPresent()) { return Errors.forException(exception.get()); diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java new file mode 100644 index 0000000000000..422414f2a9d5c --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -0,0 +1,188 @@ +/* + * 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 org.apache.kafka.server.purgatory; + +import com.yammer.metrics.core.Meter; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.server.metrics.KafkaMetricsGroup; +import org.apache.kafka.server.storage.log.FetchParams; +import org.apache.kafka.server.FetchPartitionStatus; +import org.apache.kafka.server.storage.log.FetchPartitionData; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +/** + * A remote fetch operation that can be created by the replica manager and watched + * in the remote fetch operation purgatory + */ +public class DelayedRemoteFetch extends DelayedOperation { + + private static final Logger LOG = LoggerFactory.getLogger(DelayedRemoteFetch.class); + + private static final KafkaMetricsGroup METRICS_GROUP = new KafkaMetricsGroup("kafka.server", "DelayedRemoteFetchMetrics"); + + static final Meter EXPIRED_REQUEST_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); + + private final Future remoteFetchTask; + private final CompletableFuture remoteFetchResult; + private final RemoteStorageFetchInfo remoteFetchInfo; + private final Map> fetchPartitionStatus; + private final FetchParams fetchParams; + private final Map> localReadResults; + private final Consumer partitionOrException; + private final Consumer>> responseCallback; + + public DelayedRemoteFetch(Future remoteFetchTask, + CompletableFuture remoteFetchResult, + RemoteStorageFetchInfo remoteFetchInfo, + long remoteFetchMaxWaitMs, + Map> fetchPartitionStatus, + FetchParams fetchParams, + Map> localReadResults, + Consumer partitionOrException, + Consumer>> responseCallback) { + super(remoteFetchMaxWaitMs); + this.remoteFetchTask = remoteFetchTask; + this.remoteFetchResult = remoteFetchResult; + this.remoteFetchInfo = remoteFetchInfo; + this.fetchPartitionStatus = fetchPartitionStatus; + this.fetchParams = fetchParams; + this.localReadResults = localReadResults; + this.partitionOrException = partitionOrException; + this.responseCallback = responseCallback; + + if (fetchParams.isFromFollower()) { + throw new IllegalStateException("The follower should not invoke remote fetch. Fetch params are: " + fetchParams); + } + } + + /** + * 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 + public boolean tryComplete() { + for (Map.Entry> entry : fetchPartitionStatus.entrySet()) { + TopicIdPartition topicPartition = entry.getKey(); + List fetchStatusList = entry.getValue(); + for (FetchPartitionStatus fetchStatus : fetchStatusList) { + LogOffsetMetadata fetchOffset = fetchStatus.startOffsetMetadata(); + try { + if (!fetchOffset.equals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA)) { + partitionOrException.accept(topicPartition.topicPartition()); + } + } catch (KafkaStorageException e) { // Case d + LOG.debug("Partition {} is in an offline log directory, satisfy {} immediately.", topicPartition, fetchParams); + return forceComplete(); + } catch (UnknownTopicOrPartitionException e) { // Case b + LOG.debug("Broker no longer knows of partition {}, satisfy {} immediately", topicPartition, fetchParams); + return forceComplete(); + } catch (NotLeaderOrFollowerException e) { // Case a + LOG.debug("Broker is no longer the leader or follower of {}, satisfy {} immediately", topicPartition, fetchParams); + return forceComplete(); + } + } + } + + if (remoteFetchResult.isDone()) { // Case c + return forceComplete(); + } + return false; + } + + @Override + public void onExpiration() { + // cancel the remote storage read task, if it has not been executed yet and + // avoid interrupting the task if it is already running as it may force closing opened/cached resources as transaction index. + boolean cancelled = remoteFetchTask.cancel(false); + if (!cancelled) { + LOG.debug("Remote fetch task for RemoteStorageFetchInfo: {} could not be cancelled and its isDone value is {}.", remoteFetchInfo, remoteFetchTask.isDone()); + } + + EXPIRED_REQUEST_METER.mark(); + } + + /** + * Upon completion, read whatever data is available and pass to the complete callback + */ + @Override + public void onComplete() { + Map> fetchPartitionData = new HashMap<>(); + + for (Map.Entry> entry : localReadResults.entrySet()) { + TopicIdPartition topicIdPartition = entry.getKey(); + List results = entry.getValue(); + List partitionDataList = fetchPartitionData.computeIfAbsent(topicIdPartition, k -> new ArrayList<>()); + + for (LogReadResult result : results) { + if (topicIdPartition.topicPartition().equals(remoteFetchInfo.topicPartition) + && remoteFetchResult.isDone() && result.error() == Errors.NONE + && result.info().delayedRemoteStorageFetch.isPresent()) { + + if (remoteFetchResult.get().error.isPresent()) { + partitionDataList.add(new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); + } else { + FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); + partitionDataList.add( + new FetchPartitionData( + result.error(), + result.highWatermark(), + result.leaderLogStartOffset(), + info.records, + Optional.empty(), + result.lastStableOffset(), + info.abortedTransactions, + result.preferredReadReplica(), + false)); + } + } else { + partitionDataList.add(result.toFetchPartitionData(false)); + } + } + } + + responseCallback.accept(fetchPartitionData); + } +} \ No newline at end of file From 052f60ad88d777262a8d9df6cf79c504147daa2e Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 4 Jun 2025 11:34:20 +0800 Subject: [PATCH 02/17] Move DelayedRemoteFetch to the storage module --- .../scala/kafka/server/ReplicaManager.scala | 11 +++- .../kafka/server/DelayedFetchTest.scala | 26 ++++---- .../kafka/server/DelayedRemoteFetchTest.scala | 45 +++++++------ .../AbstractCoordinatorConcurrencyTest.scala | 2 +- .../server/ReplicaManagerQuotasTest.scala | 5 +- .../kafka/server/ReplicaManagerTest.scala | 6 +- .../server/purgatory/DelayedRemoteFetch.java | 64 +++++++++++-------- 7 files changed, 93 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 8bf7b13e78e90..cd0a7e84c904a 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1592,8 +1592,15 @@ class ReplicaManager(val config: KafkaConfig, } val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong - val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, remoteFetchResult, remoteFetchInfo, remoteFetchMaxWaitMs, - fetchPartitionStatus, params, logReadResults, this, responseCallback) + val remoteFetch = new DelayedRemoteFetch(remoteFetchTask, + remoteFetchResult, + remoteFetchInfo, + remoteFetchMaxWaitMs, + fetchPartitionStatus.groupMap(_._1)(_._2).view.mapValues(_.asJava).toMap.asJava, + params, + logReadResults.groupMap(_._1)(_._2).view.mapValues(_.asJava).toMap.asJava, + tp => getPartitionOrException(tp), + response => responseCallback(response.asScala.flatMap {case (key, list) => list.asScala.map(value => (key, value))}.toSeq)) delayedRemoteFetchPurgatory.tryCompleteElseWatch(remoteFetch, util.Collections.singletonList(key)) None } diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala index f10beb0086fa8..4f1acf60130e5 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.FetchRequest -import org.apache.kafka.server.LogReadResult +import org.apache.kafka.server.{FetchPartitionStatus, LogReadResult} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogOffsetMetadata, LogOffsetSnapshot} import org.junit.jupiter.api.Test @@ -48,9 +48,9 @@ class DelayedFetchTest { val currentLeaderEpoch = Optional.of[Integer](10) val replicaId = 1 - val fetchStatus = FetchPartitionStatus( - startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + val fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -94,9 +94,9 @@ class DelayedFetchTest { val currentLeaderEpoch = Optional.of[Integer](10) val replicaId = 1 - val fetchStatus = FetchPartitionStatus( - startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + val fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(topicIdPartition.topicId(), fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -134,9 +134,9 @@ class DelayedFetchTest { val lastFetchedEpoch = Optional.of[Integer](9) val replicaId = 1 - val fetchStatus = FetchPartitionStatus( - startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch)) + val fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch, lastFetchedEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None @@ -185,9 +185,9 @@ class DelayedFetchTest { val currentLeaderEpoch = Optional.of[Integer](10) val replicaId = 1 - val fetchStatus = FetchPartitionStatus( - startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + val fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(topicIdPartition.topicId, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) val fetchParams = buildFollowerFetchParams(replicaId, maxWaitMs = 500) var fetchResultOpt: Option[FetchPartitionData] = None diff --git a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala index b65de12182ec4..19c362da2fc54 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala @@ -22,8 +22,9 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.{TopicIdPartition, Uuid} -import org.apache.kafka.server.LogReadResult +import org.apache.kafka.server.{FetchPartitionStatus, LogReadResult} import org.apache.kafka.server.metrics.KafkaYammerMetrics +import org.apache.kafka.server.purgatory.DelayedRemoteFetch import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} import org.apache.kafka.storage.internals.log._ import org.junit.jupiter.api.Assertions._ @@ -44,9 +45,9 @@ class DelayedRemoteFetchTest { private val currentLeaderEpoch = Optional.of[Integer](10) private val remoteFetchMaxWaitMs = 500 - private val fetchStatus = FetchPartitionStatus( - startOffsetMetadata = new LogOffsetMetadata(fetchOffset), - fetchInfo = new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) + private val fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) private val fetchParams = buildFetchParams(replicaId = -1, maxWaitMs = 500) @Test @@ -67,8 +68,10 @@ class DelayedRemoteFetchTest { val leaderLogStartOffset = 10 val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, - Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch( + null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), + fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), + response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) .thenReturn(mock(classOf[Partition])) @@ -103,8 +106,10 @@ class DelayedRemoteFetchTest { val leaderLogStartOffset = 10 val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) val fetchParams = buildFetchParams(replicaId = 1, maxWaitMs = 500) - assertThrows(classOf[IllegalStateException], () => new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, - Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback)) + assertThrows(classOf[IllegalStateException], () => new DelayedRemoteFetch( + null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), + fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), + response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq))) } @Test @@ -127,8 +132,10 @@ class DelayedRemoteFetchTest { val logReadInfo = buildReadResult(Errors.NONE) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, - Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch( + null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), + fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), + response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) // delayed remote fetch should still be able to complete assertTrue(delayedRemoteFetch.tryComplete()) @@ -158,8 +165,10 @@ class DelayedRemoteFetchTest { // build a read result with error val logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH) - val delayedRemoteFetch = new DelayedRemoteFetch(null, future, fetchInfo, remoteFetchMaxWaitMs, - Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch( + null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), + fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), + response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) assertTrue(delayedRemoteFetch.tryComplete()) assertTrue(delayedRemoteFetch.isCompleted) @@ -187,16 +196,14 @@ class DelayedRemoteFetchTest { val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - val delayedRemoteFetch = new DelayedRemoteFetch(remoteFetchTask, future, fetchInfo, remoteFetchMaxWaitMs, - Seq(topicIdPartition -> fetchStatus), fetchParams, Seq(topicIdPartition -> logReadInfo), replicaManager, callback) + val delayedRemoteFetch = new DelayedRemoteFetch( + remoteFetchTask, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), + fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), + response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) .thenReturn(mock(classOf[Partition])) - // Verify that the ExpiresPerSec metric is zero before fetching - val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics - assertEquals(0, metrics.keySet.asScala.count(_.getMBeanName == "kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) - // Force the delayed remote fetch to expire delayedRemoteFetch.run() @@ -204,7 +211,7 @@ class DelayedRemoteFetchTest { verify(remoteFetchTask).cancel(false) assertTrue(delayedRemoteFetch.isCompleted) - // Check that the ExpiresPerSec metric was incremented + val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics assertEquals(1, metrics.keySet.asScala.count(_.getMBeanName == "kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) // Fetch results should still include local read results diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index 2e9f95beb512e..42d9386fc1f8d 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.metadata.MetadataCache import org.apache.kafka.server.common.RequestLocal -import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets, TopicPartitionOperationKey} +import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteFetch, DelayedRemoteListOffsets, TopicPartitionOperationKey} import org.apache.kafka.server.util.timer.{MockTimer, Timer} import org.apache.kafka.server.util.{MockScheduler, MockTime, Scheduler} import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, UnifiedLog, VerificationGuard} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index a7948ae901f14..496f1b7eaa0bf 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -30,6 +30,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState +import org.apache.kafka.server.FetchPartitionStatus import org.apache.kafka.server.common.KRaftVersion import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.{KafkaScheduler, MockTime} @@ -171,7 +172,7 @@ class ReplicaManagerQuotasTest { when(partition.getReplica(1)).thenReturn(None) val tp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("t1", 0)) - val fetchPartitionStatus = FetchPartitionStatus( + val fetchPartitionStatus = new FetchPartitionStatus( new LogOffsetMetadata(50L, 0L, 250), new PartitionData(Uuid.ZERO_UUID, 50, 0, 1, Optional.empty())) val fetchParams = new FetchParams( @@ -222,7 +223,7 @@ class ReplicaManagerQuotasTest { .thenReturn(partition) val tidp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("t1", 0)) - val fetchPartitionStatus = FetchPartitionStatus( + val fetchPartitionStatus = new FetchPartitionStatus( new LogOffsetMetadata(50L, 0L, 250), new PartitionData(Uuid.ZERO_UUID, 50, 0, 1, Optional.empty())) val fetchParams = new FetchParams( diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 11a8d466bf35c..2af8aa476d6f4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -64,7 +64,7 @@ import org.apache.kafka.server.log.remote.storage._ import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.{LogReadResult, PartitionFetchState} -import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteListOffsets} +import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteFetch, DelayedRemoteListOffsets} import org.apache.kafka.server.share.SharePartitionKey import org.apache.kafka.server.share.fetch.{DelayedShareFetchGroupKey, DelayedShareFetchKey, ShareFetch} import org.apache.kafka.server.share.metrics.ShareGroupMetrics @@ -3974,13 +3974,13 @@ class ReplicaManagerTest { mock(classOf[FetchDataInfo]) }).when(spyRLM).read(any()) - val curExpiresPerSec = DelayedRemoteFetchMetrics.expiredRequestMeter.count() + val curExpiresPerSec = DelayedRemoteFetch.expiredRequestMeter().count() replicaManager.fetchMessages(params, Seq(tidp0 -> new PartitionData(topicId, fetchOffset, 0, 100000, Optional.of[Integer](leaderEpoch), Optional.of[Integer](leaderEpoch))), UNBOUNDED_QUOTA, fetchCallback) // advancing the clock to expire the delayed remote fetch timer.advanceClock(2000L) // verify the DelayedRemoteFetchMetrics.expiredRequestMeter.mark is called since the delayed remote fetch is expired - TestUtils.waitUntilTrue(() => (curExpiresPerSec + 1) == DelayedRemoteFetchMetrics.expiredRequestMeter.count(), "DelayedRemoteFetchMetrics.expiredRequestMeter.count() should be 1, but got: " + DelayedRemoteFetchMetrics.expiredRequestMeter.count(), 10000L) + TestUtils.waitUntilTrue(() => (curExpiresPerSec + 1) == DelayedRemoteFetch.expiredRequestMeter().count(), "DelayedRemoteFetchMetrics.expiredRequestMeter.count() should be 1, but got: " + DelayedRemoteFetch.expiredRequestMeter().count(), 10000L) latch.countDown() } finally { Utils.tryAll(util.Arrays.asList[Callable[Void]]( diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 422414f2a9d5c..488eb912f6ae8 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -43,6 +43,7 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -151,38 +152,49 @@ public void onExpiration() { public void onComplete() { Map> fetchPartitionData = new HashMap<>(); - for (Map.Entry> entry : localReadResults.entrySet()) { - TopicIdPartition topicIdPartition = entry.getKey(); - List results = entry.getValue(); - List partitionDataList = fetchPartitionData.computeIfAbsent(topicIdPartition, k -> new ArrayList<>()); - - for (LogReadResult result : results) { - if (topicIdPartition.topicPartition().equals(remoteFetchInfo.topicPartition) - && remoteFetchResult.isDone() && result.error() == Errors.NONE - && result.info().delayedRemoteStorageFetch.isPresent()) { - - if (remoteFetchResult.get().error.isPresent()) { - partitionDataList.add(new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); + try { + for (Map.Entry> entry : localReadResults.entrySet()) { + TopicIdPartition topicIdPartition = entry.getKey(); + List results = entry.getValue(); + List partitionDataList = fetchPartitionData.computeIfAbsent(topicIdPartition, + k -> new ArrayList<>()); + + for (LogReadResult result : results) { + if (topicIdPartition.topicPartition().equals(remoteFetchInfo.topicPartition) + && remoteFetchResult.isDone() && result.error() == Errors.NONE + && result.info().delayedRemoteStorageFetch.isPresent()) { + + if (remoteFetchResult.get().error.isPresent()) { + partitionDataList.add( + new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); + } else { + FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); + partitionDataList.add( + new FetchPartitionData( + result.error(), + result.highWatermark(), + result.leaderLogStartOffset(), + info.records, + Optional.empty(), + result.lastStableOffset(), + info.abortedTransactions, + result.preferredReadReplica(), + false)); + } } else { - FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); - partitionDataList.add( - new FetchPartitionData( - result.error(), - result.highWatermark(), - result.leaderLogStartOffset(), - info.records, - Optional.empty(), - result.lastStableOffset(), - info.abortedTransactions, - result.preferredReadReplica(), - false)); + partitionDataList.add(result.toFetchPartitionData(false)); } - } else { - partitionDataList.add(result.toFetchPartitionData(false)); } } + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); } responseCallback.accept(fetchPartitionData); } + + // Visible for testing + public static Meter expiredRequestMeter() { + return EXPIRED_REQUEST_METER; + } } \ No newline at end of file From 37aa96fce1a414312566e94e629d411af02f2c77 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 4 Jun 2025 15:06:31 +0800 Subject: [PATCH 03/17] move ut --- .../kafka/server/DelayedRemoteFetchTest.scala | 256 -------------- .../purgatory/DelayedRemoteFetchTest.java | 313 ++++++++++++++++++ 2 files changed, 313 insertions(+), 256 deletions(-) delete mode 100644 core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala create mode 100644 server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java diff --git a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala deleted file mode 100644 index 19c362da2fc54..0000000000000 --- a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala +++ /dev/null @@ -1,256 +0,0 @@ -/* - * 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.cluster.Partition -import org.apache.kafka.common.errors.NotLeaderOrFollowerException -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.MemoryRecords -import org.apache.kafka.common.requests.FetchRequest -import org.apache.kafka.common.{TopicIdPartition, Uuid} -import org.apache.kafka.server.{FetchPartitionStatus, LogReadResult} -import org.apache.kafka.server.metrics.KafkaYammerMetrics -import org.apache.kafka.server.purgatory.DelayedRemoteFetch -import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} -import org.apache.kafka.storage.internals.log._ -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test -import org.mockito.Mockito.{mock, verify, when} - -import java.util.{Optional, OptionalLong} -import java.util.concurrent.{CompletableFuture, Future} -import scala.collection._ -import scala.jdk.CollectionConverters._ - -class DelayedRemoteFetchTest { - private val maxBytes = 1024 - private val replicaManager: ReplicaManager = mock(classOf[ReplicaManager]) - private val topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic") - private val fetchOffset = 500L - private val logStartOffset = 0L - private val currentLeaderEpoch = Optional.of[Integer](10) - private val remoteFetchMaxWaitMs = 500 - - private val fetchStatus = new FetchPartitionStatus( - new LogOffsetMetadata(fetchOffset), - new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)) - private val fetchParams = buildFetchParams(replicaId = -1, maxWaitMs = 500) - - @Test - def testFetch(): Unit = { - var actualTopicPartition: Option[TopicIdPartition] = None - var fetchResultOpt: Option[FetchPartitionData] = None - - def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { - assertEquals(1, responses.size) - actualTopicPartition = Some(responses.head._1) - fetchResultOpt = Some(responses.head._2) - } - - val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]() - future.complete(null) - val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) - val highWatermark = 100 - val leaderLogStartOffset = 10 - val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - - val delayedRemoteFetch = new DelayedRemoteFetch( - null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), - fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), - response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) - - when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) - .thenReturn(mock(classOf[Partition])) - - assertTrue(delayedRemoteFetch.tryComplete()) - assertTrue(delayedRemoteFetch.isCompleted) - assertTrue(actualTopicPartition.isDefined) - assertEquals(topicIdPartition, actualTopicPartition.get) - assertTrue(fetchResultOpt.isDefined) - - val fetchResult = fetchResultOpt.get - assertEquals(Errors.NONE, fetchResult.error) - assertEquals(highWatermark, fetchResult.highWatermark) - assertEquals(leaderLogStartOffset, fetchResult.logStartOffset) - } - - @Test - def testFollowerFetch(): Unit = { - var actualTopicPartition: Option[TopicIdPartition] = None - var fetchResultOpt: Option[FetchPartitionData] = None - - def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { - assertEquals(1, responses.size) - actualTopicPartition = Some(responses.head._1) - fetchResultOpt = Some(responses.head._2) - } - - val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]() - future.complete(null) - val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) - val highWatermark = 100 - val leaderLogStartOffset = 10 - val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - val fetchParams = buildFetchParams(replicaId = 1, maxWaitMs = 500) - assertThrows(classOf[IllegalStateException], () => new DelayedRemoteFetch( - null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), - fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), - response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq))) - } - - @Test - def testNotLeaderOrFollower(): Unit = { - var actualTopicPartition: Option[TopicIdPartition] = None - var fetchResultOpt: Option[FetchPartitionData] = None - - def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { - assertEquals(1, responses.size) - actualTopicPartition = Some(responses.head._1) - fetchResultOpt = Some(responses.head._2) - } - - // throw exception while getPartition - when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) - .thenThrow(new NotLeaderOrFollowerException(s"Replica for $topicIdPartition not available")) - - val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]() - val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) - - val logReadInfo = buildReadResult(Errors.NONE) - - val delayedRemoteFetch = new DelayedRemoteFetch( - null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), - fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), - response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) - - // delayed remote fetch should still be able to complete - assertTrue(delayedRemoteFetch.tryComplete()) - assertTrue(delayedRemoteFetch.isCompleted) - assertEquals(topicIdPartition, actualTopicPartition.get) - assertTrue(fetchResultOpt.isDefined) - } - - @Test - def testErrorLogReadInfo(): Unit = { - var actualTopicPartition: Option[TopicIdPartition] = None - var fetchResultOpt: Option[FetchPartitionData] = None - - def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { - assertEquals(1, responses.size) - actualTopicPartition = Some(responses.head._1) - fetchResultOpt = Some(responses.head._2) - } - - when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) - .thenReturn(mock(classOf[Partition])) - - val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]() - future.complete(null) - val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) - - // build a read result with error - val logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH) - - val delayedRemoteFetch = new DelayedRemoteFetch( - null, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), - fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), - response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) - - assertTrue(delayedRemoteFetch.tryComplete()) - assertTrue(delayedRemoteFetch.isCompleted) - assertEquals(topicIdPartition, actualTopicPartition.get) - assertTrue(fetchResultOpt.isDefined) - assertEquals(Errors.FENCED_LEADER_EPOCH, fetchResultOpt.get.error) - } - - @Test - def testRequestExpiry(): Unit = { - var actualTopicPartition: Option[TopicIdPartition] = None - var fetchResultOpt: Option[FetchPartitionData] = None - - def callback(responses: Seq[(TopicIdPartition, FetchPartitionData)]): Unit = { - assertEquals(1, responses.size) - actualTopicPartition = Some(responses.head._1) - fetchResultOpt = Some(responses.head._2) - } - - val highWatermark = 100 - val leaderLogStartOffset = 10 - - val remoteFetchTask = mock(classOf[Future[Void]]) - val future: CompletableFuture[RemoteLogReadResult] = new CompletableFuture[RemoteLogReadResult]() - val fetchInfo: RemoteStorageFetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition.topicPartition(), null, null) - val logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset) - - val delayedRemoteFetch = new DelayedRemoteFetch( - remoteFetchTask, future, fetchInfo, remoteFetchMaxWaitMs, java.util.Map.of(topicIdPartition, java.util.List.of(fetchStatus)), - fetchParams, java.util.Map.of(topicIdPartition, java.util.List.of(logReadInfo)), tp => replicaManager.getPartitionOrException(tp), - response => callback(response.asScala.flatMap { case (key, list) => list.asScala.map(value => (key, value)) }.toSeq)) - - when(replicaManager.getPartitionOrException(topicIdPartition.topicPartition)) - .thenReturn(mock(classOf[Partition])) - - // Force the delayed remote fetch to expire - delayedRemoteFetch.run() - - // Check that the task was cancelled and force-completed - verify(remoteFetchTask).cancel(false) - assertTrue(delayedRemoteFetch.isCompleted) - - val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics - assertEquals(1, metrics.keySet.asScala.count(_.getMBeanName == "kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) - - // Fetch results should still include local read results - assertTrue(actualTopicPartition.isDefined) - assertEquals(topicIdPartition, actualTopicPartition.get) - assertTrue(fetchResultOpt.isDefined) - - val fetchResult = fetchResultOpt.get - assertEquals(Errors.NONE, fetchResult.error) - assertEquals(highWatermark, fetchResult.highWatermark) - assertEquals(leaderLogStartOffset, fetchResult.logStartOffset) - } - - private def buildFetchParams(replicaId: Int, - maxWaitMs: Int): FetchParams = { - new FetchParams( - replicaId, - 1, - maxWaitMs, - 1, - maxBytes, - FetchIsolation.LOG_END, - Optional.empty() - ) - } - - private def buildReadResult(error: Errors, - highWatermark: Int = 0, - leaderLogStartOffset: Int = 0): LogReadResult = { - new LogReadResult( - new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), - Optional.empty(), - highWatermark, - leaderLogStartOffset, - -1L, - -1L, - -1L, - OptionalLong.empty(), - if (error != Errors.NONE) Optional.of[Throwable](error.exception) else Optional.empty[Throwable]()) - } - -} diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java new file mode 100644 index 0000000000000..452468e5fcf68 --- /dev/null +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -0,0 +1,313 @@ +/* + * 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 org.apache.kafka.server.purgatory; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.requests.FetchRequest; +import org.apache.kafka.server.FetchPartitionStatus; +import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.server.metrics.KafkaYammerMetrics; +import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.storage.log.FetchParams; +import org.apache.kafka.server.storage.log.FetchPartitionData; +import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +public class DelayedRemoteFetchTest { + private final int maxBytes = 1024; + private final Consumer partitionOrException = mock(Consumer.class); + private final TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic"); + private final long fetchOffset = 500L; + private final long logStartOffset = 0L; + private final Optional currentLeaderEpoch = Optional.of(10); + private final int remoteFetchMaxWaitMs = 500; + + private final FetchPartitionStatus fetchStatus = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch) + ); + private final FetchParams fetchParams = buildFetchParams(-1, 500); + + @Test + public void testFetch() { + AtomicReference actualTopicPartition = new AtomicReference<>(); + AtomicReference fetchResultOpt = new AtomicReference<>(); + + Consumer>> callback = responses -> { + assertEquals(1, responses.size()); + Map.Entry> entry = responses.entrySet().iterator().next(); + actualTopicPartition.set(entry.getKey()); + fetchResultOpt.set(entry.getValue().get(0)); + }; + + CompletableFuture future = new CompletableFuture<>(); + future.complete(null); + + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, + topicIdPartition.topicPartition(), null, null); + int highWatermark = 100; + int leaderLogStartOffset = 10; + LogReadResult logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + null, + future, + fetchInfo, + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, List.of(fetchStatus)), + fetchParams, + Map.of(topicIdPartition, List.of(logReadInfo)), + partitionOrException, + callback + ); + + assertTrue(delayedRemoteFetch.tryComplete()); + assertTrue(delayedRemoteFetch.isCompleted()); + assertNotNull(actualTopicPartition.get()); + assertEquals(topicIdPartition, actualTopicPartition.get()); + assertNotNull(fetchResultOpt.get()); + + FetchPartitionData fetchResult = fetchResultOpt.get(); + assertEquals(Errors.NONE, fetchResult.error); + assertEquals(highWatermark, fetchResult.highWatermark); + assertEquals(leaderLogStartOffset, fetchResult.logStartOffset); + } + + @Test + public void testFollowerFetch() { + AtomicReference actualTopicPartition = new AtomicReference<>(); + AtomicReference fetchResultOpt = new AtomicReference<>(); + + Consumer>> callback = responses -> { + assertEquals(1, responses.size()); + Map.Entry> entry = responses.entrySet().iterator().next(); + actualTopicPartition.set(entry.getKey()); + fetchResultOpt.set(entry.getValue().get(0)); + }; + + CompletableFuture future = new CompletableFuture<>(); + future.complete(null); + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, + new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + LogReadResult logReadInfo = buildReadResult(Errors.NONE, 100, 10); + + assertThrows(IllegalStateException.class, () -> + new DelayedRemoteFetch( + null, + future, + fetchInfo, + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, List.of(fetchStatus)), + buildFetchParams(1, 500), + Map.of(topicIdPartition, List.of(logReadInfo)), + partitionOrException, + callback + )); + } + + @Test + public void testNotLeaderOrFollower() { + AtomicReference actualTopicPartition = new AtomicReference<>(); + AtomicReference fetchResultOpt = new AtomicReference<>(); + + Consumer>> callback = responses -> { + assertEquals(1, responses.size()); + Map.Entry> entry = responses.entrySet().iterator().next(); + actualTopicPartition.set(entry.getKey()); + fetchResultOpt.set(entry.getValue().get(0)); + }; + + // throw exception while getPartition + doThrow(new NotLeaderOrFollowerException(String.format("Replica for %s not available", topicIdPartition))) + .when(partitionOrException).accept(topicIdPartition.topicPartition()); + + CompletableFuture future = new CompletableFuture<>(); + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, + new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + + LogReadResult logReadInfo = buildReadResult(Errors.NONE); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + null, + future, + fetchInfo, + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, List.of(fetchStatus)), + fetchParams, + Map.of(topicIdPartition, List.of(logReadInfo)), + partitionOrException, + callback); + + // delayed remote fetch should still be able to complete + assertTrue(delayedRemoteFetch.tryComplete()); + assertTrue(delayedRemoteFetch.isCompleted()); + assertEquals(topicIdPartition, actualTopicPartition.get()); + assertNotNull(fetchResultOpt.get()); + } + + @Test + public void testErrorLogReadInfo() { + AtomicReference actualTopicPartition = new AtomicReference<>(); + AtomicReference fetchResultOpt = new AtomicReference<>(); + + Consumer>> callback = responses -> { + assertEquals(1, responses.size()); + Map.Entry> entry = responses.entrySet().iterator().next(); + actualTopicPartition.set(entry.getKey()); + fetchResultOpt.set(entry.getValue().get(0)); + }; + + CompletableFuture future = new CompletableFuture<>(); + future.complete(null); + + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, + new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + + // build a read result with error + LogReadResult logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + null, + future, + fetchInfo, + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, List.of(fetchStatus)), + fetchParams, + Map.of(topicIdPartition, List.of(logReadInfo)), + partitionOrException, + callback + ); + + assertTrue(delayedRemoteFetch.tryComplete()); + assertTrue(delayedRemoteFetch.isCompleted()); + assertEquals(topicIdPartition, actualTopicPartition.get()); + assertNotNull(fetchResultOpt.get()); + assertEquals(Errors.FENCED_LEADER_EPOCH, fetchResultOpt.get().error); + } + + @Test + public void testRequestExpiry() { + AtomicReference actualTopicPartition = new AtomicReference<>(); + AtomicReference fetchResultOpt = new AtomicReference<>(); + + Consumer>> callback = responses -> { + assertEquals(1, responses.size()); + Map.Entry> entry = responses.entrySet().iterator().next(); + actualTopicPartition.set(entry.getKey()); + fetchResultOpt.set(entry.getValue().get(0)); + }; + + int highWatermark = 100; + int leaderLogStartOffset = 10; + + Future remoteFetchTask = mock(Future.class); + CompletableFuture future = new CompletableFuture<>(); + + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, + new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + LogReadResult logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + remoteFetchTask, + future, + fetchInfo, + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, List.of(fetchStatus)), + fetchParams, + Map.of(topicIdPartition, List.of(logReadInfo)), + partitionOrException, + callback + ); + + // Force the delayed remote fetch to expire + delayedRemoteFetch.run(); + + // Check that the task was cancelled and force-completed + verify(remoteFetchTask).cancel(false); + assertTrue(delayedRemoteFetch.isCompleted()); + + long metricsCount = KafkaYammerMetrics.defaultRegistry().allMetrics().keySet().stream() + .filter(m -> m.getMBeanName().equals("kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) + .count(); + assertEquals(1, metricsCount); + + // Fetch results should still include local read results + assertNotNull(actualTopicPartition.get()); + assertEquals(topicIdPartition, actualTopicPartition.get()); + assertNotNull(fetchResultOpt.get()); + + FetchPartitionData fetchResult = fetchResultOpt.get(); + assertEquals(Errors.NONE, fetchResult.error); + assertEquals(highWatermark, fetchResult.highWatermark); + assertEquals(leaderLogStartOffset, fetchResult.logStartOffset); + } + + private FetchParams buildFetchParams(int replicaId, int maxWaitMs) { + return new FetchParams( + replicaId, + 1, + maxWaitMs, + 1, + maxBytes, + FetchIsolation.LOG_END, + Optional.empty() + ); + } + + private LogReadResult buildReadResult(Errors error) { + return buildReadResult(error, 0, 0); + } + + private LogReadResult buildReadResult(Errors error, int highWatermark, int leaderLogStartOffset) { + return new LogReadResult( + new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), + Optional.empty(), + highWatermark, + leaderLogStartOffset, + -1L, + -1L, + -1L, + OptionalLong.empty(), + error != Errors.NONE ? Optional.of(error.exception()) : Optional.empty()); + } +} \ No newline at end of file From e5d1abbb94602e65f4fbfe2a3224f2f4ea6dc6ef Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Thu, 5 Jun 2025 23:18:55 +0800 Subject: [PATCH 04/17] fix import --- .../server/purgatory/DelayedRemoteFetch.java | 7 ++++--- .../purgatory/DelayedRemoteFetchTest.java | 18 +++++++++--------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 488eb912f6ae8..bc1e858bb58a1 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -17,23 +17,24 @@ package org.apache.kafka.server.purgatory; -import com.yammer.metrics.core.Meter; - import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.KafkaStorageException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.server.FetchPartitionStatus; import org.apache.kafka.server.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.storage.log.FetchParams; -import org.apache.kafka.server.FetchPartitionStatus; import org.apache.kafka.server.storage.log.FetchPartitionData; import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; + +import com.yammer.metrics.core.Meter; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index 452468e5fcf68..6dfd10e73d8bc 100644 --- a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -14,17 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.server.purgatory; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; - import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; @@ -42,6 +33,7 @@ import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; + import org.junit.jupiter.api.Test; import java.util.List; @@ -53,6 +45,14 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + public class DelayedRemoteFetchTest { private final int maxBytes = 1024; private final Consumer partitionOrException = mock(Consumer.class); From 52156c0be62d295f9cf3a63dac5be57d6634af9d Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 22 Jul 2025 11:58:32 +0800 Subject: [PATCH 05/17] fix conflict with KAFKA-14915 --- .../scala/kafka/server/ReplicaManager.scala | 11 +- .../server/purgatory/DelayedRemoteFetch.java | 138 ++++--- .../purgatory/DelayedRemoteFetchTest.java | 343 +++++++++++++----- 3 files changed, 331 insertions(+), 161 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 2088335f018ee..073ff571e441c 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1628,8 +1628,15 @@ class ReplicaManager(val config: KafkaConfig, } val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong - val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs, - remoteFetchPartitionStatus, params, logReadResults, this, responseCallback) + val remoteFetch = new DelayedRemoteFetch(remoteFetchTasks, + remoteFetchResults, + remoteFetchInfos, + remoteFetchMaxWaitMs, + remoteFetchPartitionStatus.toMap.asJava, + params, + logReadResults.toMap.asJava, + tp => getPartitionOrException(tp), + response => responseCallback(response.asScala.toSeq)) // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation val delayedFetchKeys = remoteFetchPartitionStatus.map { case (tp, _) => new TopicPartitionOperationKey(tp) }.toList diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index bc1e858bb58a1..32e27f78fc867 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -38,9 +38,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -61,28 +59,28 @@ public class DelayedRemoteFetch extends DelayedOperation { static final Meter EXPIRED_REQUEST_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); - private final Future remoteFetchTask; - private final CompletableFuture remoteFetchResult; - private final RemoteStorageFetchInfo remoteFetchInfo; - private final Map> fetchPartitionStatus; + private final Map> remoteFetchTasks; + private final Map> remoteFetchResults; + private final Map remoteFetchInfos; + private final Map fetchPartitionStatus; private final FetchParams fetchParams; - private final Map> localReadResults; + private final Map localReadResults; private final Consumer partitionOrException; - private final Consumer>> responseCallback; + private final Consumer> responseCallback; - public DelayedRemoteFetch(Future remoteFetchTask, - CompletableFuture remoteFetchResult, - RemoteStorageFetchInfo remoteFetchInfo, + public DelayedRemoteFetch(Map> remoteFetchTasks, + Map> remoteFetchResults, + Map remoteFetchInfos, long remoteFetchMaxWaitMs, - Map> fetchPartitionStatus, + Map fetchPartitionStatus, FetchParams fetchParams, - Map> localReadResults, + Map localReadResults, Consumer partitionOrException, - Consumer>> responseCallback) { + Consumer> responseCallback) { super(remoteFetchMaxWaitMs); - this.remoteFetchTask = remoteFetchTask; - this.remoteFetchResult = remoteFetchResult; - this.remoteFetchInfo = remoteFetchInfo; + this.remoteFetchTasks = remoteFetchTasks; + this.remoteFetchResults = remoteFetchResults; + this.remoteFetchInfos = remoteFetchInfos; this.fetchPartitionStatus = fetchPartitionStatus; this.fetchParams = fetchParams; this.localReadResults = localReadResults; @@ -99,36 +97,35 @@ public DelayedRemoteFetch(Future remoteFetchTask, * * 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 c: All 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 public boolean tryComplete() { - for (Map.Entry> entry : fetchPartitionStatus.entrySet()) { + for (Map.Entry entry : fetchPartitionStatus.entrySet()) { TopicIdPartition topicPartition = entry.getKey(); - List fetchStatusList = entry.getValue(); - for (FetchPartitionStatus fetchStatus : fetchStatusList) { - LogOffsetMetadata fetchOffset = fetchStatus.startOffsetMetadata(); - try { - if (!fetchOffset.equals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA)) { - partitionOrException.accept(topicPartition.topicPartition()); - } - } catch (KafkaStorageException e) { // Case d - LOG.debug("Partition {} is in an offline log directory, satisfy {} immediately.", topicPartition, fetchParams); - return forceComplete(); - } catch (UnknownTopicOrPartitionException e) { // Case b - LOG.debug("Broker no longer knows of partition {}, satisfy {} immediately", topicPartition, fetchParams); - return forceComplete(); - } catch (NotLeaderOrFollowerException e) { // Case a - LOG.debug("Broker is no longer the leader or follower of {}, satisfy {} immediately", topicPartition, fetchParams); - return forceComplete(); + FetchPartitionStatus fetchStatus = entry.getValue(); + LogOffsetMetadata fetchOffset = fetchStatus.startOffsetMetadata(); + try { + if (!fetchOffset.equals(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA)) { + partitionOrException.accept(topicPartition.topicPartition()); } + } catch (KafkaStorageException e) { // Case d + LOG.debug("Partition {} is in an offline log directory, satisfy {} immediately.", topicPartition, fetchParams); + return forceComplete(); + } catch (UnknownTopicOrPartitionException e) { // Case b + LOG.debug("Broker no longer knows of partition {}, satisfy {} immediately", topicPartition, fetchParams); + return forceComplete(); + } catch (NotLeaderOrFollowerException e) { // Case a + LOG.debug("Broker is no longer the leader or follower of {}, satisfy {} immediately", topicPartition, fetchParams); + return forceComplete(); } } - if (remoteFetchResult.isDone()) { // Case c + // Case c + if (remoteFetchResults.values().stream().allMatch(CompletableFuture::isDone)) { return forceComplete(); } return false; @@ -138,10 +135,13 @@ public boolean tryComplete() { public void onExpiration() { // cancel the remote storage read task, if it has not been executed yet and // avoid interrupting the task if it is already running as it may force closing opened/cached resources as transaction index. - boolean cancelled = remoteFetchTask.cancel(false); - if (!cancelled) { - LOG.debug("Remote fetch task for RemoteStorageFetchInfo: {} could not be cancelled and its isDone value is {}.", remoteFetchInfo, remoteFetchTask.isDone()); - } + remoteFetchTasks.forEach(((topicIdPartition, task) -> { + if (task != null && !task.isDone()) { + if (!task.cancel(false)) { + LOG.debug("Remote fetch task for remoteFetchInfo: {} could not be cancelled.", remoteFetchInfos.get(topicIdPartition)); + } + } + })); EXPIRED_REQUEST_METER.mark(); } @@ -151,40 +151,36 @@ public void onExpiration() { */ @Override public void onComplete() { - Map> fetchPartitionData = new HashMap<>(); - + Map fetchPartitionData = new HashMap<>(); try { - for (Map.Entry> entry : localReadResults.entrySet()) { - TopicIdPartition topicIdPartition = entry.getKey(); - List results = entry.getValue(); - List partitionDataList = fetchPartitionData.computeIfAbsent(topicIdPartition, - k -> new ArrayList<>()); - - for (LogReadResult result : results) { - if (topicIdPartition.topicPartition().equals(remoteFetchInfo.topicPartition) - && remoteFetchResult.isDone() && result.error() == Errors.NONE - && result.info().delayedRemoteStorageFetch.isPresent()) { - - if (remoteFetchResult.get().error.isPresent()) { - partitionDataList.add( - new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); - } else { - FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); - partitionDataList.add( - new FetchPartitionData( - result.error(), - result.highWatermark(), - result.leaderLogStartOffset(), - info.records, - Optional.empty(), - result.lastStableOffset(), - info.abortedTransactions, - result.preferredReadReplica(), - false)); - } + for (Map.Entry entry : localReadResults.entrySet()) { + TopicIdPartition tp = entry.getKey(); + LogReadResult result = entry.getValue(); + + CompletableFuture remoteFetchResult = remoteFetchResults.get(tp); + if (remoteFetchInfos.containsKey(tp) + && remoteFetchResult.isDone() && result.error() == Errors.NONE + && result.info().delayedRemoteStorageFetch.isPresent()) { + + if (remoteFetchResult.get().error.isPresent()) { + fetchPartitionData.put(tp, + new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); } else { - partitionDataList.add(result.toFetchPartitionData(false)); + FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); + fetchPartitionData.put(tp, + new FetchPartitionData( + result.error(), + result.highWatermark(), + result.leaderLogStartOffset(), + info.records, + Optional.empty(), + result.lastStableOffset(), + info.abortedTransactions, + result.preferredReadReplica(), + false)); } + } else { + fetchPartitionData.put(tp, result.toFetchPartitionData(false)); } } } catch (InterruptedException | ExecutionException e) { diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index 6dfd10e73d8bc..878c1ed37a95b 100644 --- a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -34,9 +34,13 @@ import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; +import com.yammer.metrics.core.Meter; +import com.yammer.metrics.core.Metric; +import com.yammer.metrics.core.MetricName; + import org.junit.jupiter.api.Test; -import java.util.List; +import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; @@ -46,17 +50,22 @@ import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class DelayedRemoteFetchTest { private final int maxBytes = 1024; private final Consumer partitionOrException = mock(Consumer.class); private final TopicIdPartition topicIdPartition = new TopicIdPartition(Uuid.randomUuid(), 0, "topic"); + private final TopicIdPartition topicIdPartition2 = new TopicIdPartition(Uuid.randomUuid(), 0, "topic2"); private final long fetchOffset = 500L; private final long logStartOffset = 0L; private final Optional currentLeaderEpoch = Optional.of(10); @@ -73,30 +82,29 @@ public void testFetch() { AtomicReference actualTopicPartition = new AtomicReference<>(); AtomicReference fetchResultOpt = new AtomicReference<>(); - Consumer>> callback = responses -> { + Consumer> callback = responses -> { assertEquals(1, responses.size()); - Map.Entry> entry = responses.entrySet().iterator().next(); + Map.Entry entry = responses.entrySet().iterator().next(); actualTopicPartition.set(entry.getKey()); - fetchResultOpt.set(entry.getValue().get(0)); + fetchResultOpt.set(entry.getValue()); }; CompletableFuture future = new CompletableFuture<>(); - future.complete(null); + future.complete(buildRemoteReadResult(Errors.NONE)); - RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, - topicIdPartition.topicPartition(), null, null); - int highWatermark = 100; - int leaderLogStartOffset = 10; + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + long highWatermark = 100L; + long leaderLogStartOffset = 10L; LogReadResult logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset); DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( - null, - future, - fetchInfo, + Map.of(), + Map.of(topicIdPartition, future), + Map.of(topicIdPartition, fetchInfo), remoteFetchMaxWaitMs, - Map.of(topicIdPartition, List.of(fetchStatus)), + Map.of(topicIdPartition, fetchStatus), fetchParams, - Map.of(topicIdPartition, List.of(logReadInfo)), + Map.of(topicIdPartition, logReadInfo), partitionOrException, callback ); @@ -115,31 +123,24 @@ public void testFetch() { @Test public void testFollowerFetch() { - AtomicReference actualTopicPartition = new AtomicReference<>(); - AtomicReference fetchResultOpt = new AtomicReference<>(); - - Consumer>> callback = responses -> { + Consumer> callback = responses -> { assertEquals(1, responses.size()); - Map.Entry> entry = responses.entrySet().iterator().next(); - actualTopicPartition.set(entry.getKey()); - fetchResultOpt.set(entry.getValue().get(0)); }; CompletableFuture future = new CompletableFuture<>(); - future.complete(null); - RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, - new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); - LogReadResult logReadInfo = buildReadResult(Errors.NONE, 100, 10); + future.complete(buildRemoteReadResult(Errors.NONE)); + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + LogReadResult logReadInfo = buildReadResult(Errors.NONE, 100L, 10L); assertThrows(IllegalStateException.class, () -> new DelayedRemoteFetch( - null, - future, - fetchInfo, + Map.of(), + Map.of(topicIdPartition, future), + Map.of(topicIdPartition, fetchInfo), remoteFetchMaxWaitMs, - Map.of(topicIdPartition, List.of(fetchStatus)), + Map.of(topicIdPartition, fetchStatus), buildFetchParams(1, 500), - Map.of(topicIdPartition, List.of(logReadInfo)), + Map.of(topicIdPartition, logReadInfo), partitionOrException, callback )); @@ -150,11 +151,11 @@ public void testNotLeaderOrFollower() { AtomicReference actualTopicPartition = new AtomicReference<>(); AtomicReference fetchResultOpt = new AtomicReference<>(); - Consumer>> callback = responses -> { + Consumer> callback = responses -> { assertEquals(1, responses.size()); - Map.Entry> entry = responses.entrySet().iterator().next(); + Map.Entry entry = responses.entrySet().iterator().next(); actualTopicPartition.set(entry.getKey()); - fetchResultOpt.set(entry.getValue().get(0)); + fetchResultOpt.set(entry.getValue()); }; // throw exception while getPartition @@ -162,21 +163,21 @@ public void testNotLeaderOrFollower() { .when(partitionOrException).accept(topicIdPartition.topicPartition()); CompletableFuture future = new CompletableFuture<>(); - RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, - new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); LogReadResult logReadInfo = buildReadResult(Errors.NONE); DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( - null, - future, - fetchInfo, + Map.of(), + Map.of(topicIdPartition, future), + Map.of(topicIdPartition, fetchInfo), remoteFetchMaxWaitMs, - Map.of(topicIdPartition, List.of(fetchStatus)), + Map.of(topicIdPartition, fetchStatus), fetchParams, - Map.of(topicIdPartition, List.of(logReadInfo)), + Map.of(topicIdPartition, logReadInfo), partitionOrException, - callback); + callback + ); // delayed remote fetch should still be able to complete assertTrue(delayedRemoteFetch.tryComplete()); @@ -190,30 +191,29 @@ public void testErrorLogReadInfo() { AtomicReference actualTopicPartition = new AtomicReference<>(); AtomicReference fetchResultOpt = new AtomicReference<>(); - Consumer>> callback = responses -> { + Consumer> callback = responses -> { assertEquals(1, responses.size()); - Map.Entry> entry = responses.entrySet().iterator().next(); + Map.Entry entry = responses.entrySet().iterator().next(); actualTopicPartition.set(entry.getKey()); - fetchResultOpt.set(entry.getValue().get(0)); + fetchResultOpt.set(entry.getValue()); }; CompletableFuture future = new CompletableFuture<>(); - future.complete(null); + future.complete(buildRemoteReadResult(Errors.NONE)); - RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, - new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); + RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); // build a read result with error LogReadResult logReadInfo = buildReadResult(Errors.FENCED_LEADER_EPOCH); DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( - null, - future, - fetchInfo, + Map.of(), + Map.of(topicIdPartition, future), + Map.of(topicIdPartition, fetchInfo), remoteFetchMaxWaitMs, - Map.of(topicIdPartition, List.of(fetchStatus)), + Map.of(topicIdPartition, fetchStatus), fetchParams, - Map.of(topicIdPartition, List.of(logReadInfo)), + Map.of(topicIdPartition, logReadInfo), partitionOrException, callback ); @@ -225,61 +225,221 @@ public void testErrorLogReadInfo() { assertEquals(Errors.FENCED_LEADER_EPOCH, fetchResultOpt.get().error); } + private long expiresPerSecValue() { + Map allMetrics = KafkaYammerMetrics.defaultRegistry().allMetrics(); + return allMetrics.entrySet() + .stream() + .filter(e -> e.getKey().getMBeanName().endsWith("kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) + .findFirst() + .map(Map.Entry::getValue) + .filter(Meter.class::isInstance) + .map(Meter.class::cast) + .map(Meter::count) + .orElse(0L); + } + @Test public void testRequestExpiry() { - AtomicReference actualTopicPartition = new AtomicReference<>(); - AtomicReference fetchResultOpt = new AtomicReference<>(); + Map responses = new HashMap<>(); - Consumer>> callback = responses -> { - assertEquals(1, responses.size()); - Map.Entry> entry = responses.entrySet().iterator().next(); - actualTopicPartition.set(entry.getKey()); - fetchResultOpt.set(entry.getValue().get(0)); - }; + Consumer> callback = responses::putAll; - int highWatermark = 100; - int leaderLogStartOffset = 10; + Future remoteFetchTaskExpired = mock(Future.class); + Future remoteFetchTask2 = mock(Future.class); + // complete the 2nd task, and keep the 1st one expired + when(remoteFetchTask2.isDone()).thenReturn(true); - Future remoteFetchTask = mock(Future.class); - CompletableFuture future = new CompletableFuture<>(); + // Create futures - one completed, one not + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + // Only complete one remote fetch + future2.complete(buildRemoteReadResult(Errors.NONE)); - RemoteStorageFetchInfo fetchInfo = new RemoteStorageFetchInfo(0, false, - new TopicPartition(topicIdPartition.topic(), topicIdPartition.partition()), null, null); - LogReadResult logReadInfo = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset); + RemoteStorageFetchInfo fetchInfo1 = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + RemoteStorageFetchInfo fetchInfo2 = new RemoteStorageFetchInfo(0, false, topicIdPartition2, null, null); + + long highWatermark = 100L; + long leaderLogStartOffset = 10L; + + LogReadResult logReadInfo1 = buildReadResult(Errors.NONE, highWatermark, leaderLogStartOffset); + LogReadResult logReadInfo2 = buildReadResult(Errors.NONE); + + FetchPartitionStatus fetchStatus1 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)); + + FetchPartitionStatus fetchStatus2 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset + 100L), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset + 100L, logStartOffset, maxBytes, currentLeaderEpoch)); + + // Set up maps for multiple partitions + Map> remoteFetchTasks = Map.of(topicIdPartition, remoteFetchTaskExpired, topicIdPartition2, remoteFetchTask2); + Map> remoteFetchResults = Map.of(topicIdPartition, future1, topicIdPartition2, future2); + Map remoteFetchInfos = Map.of(topicIdPartition, fetchInfo1, topicIdPartition2, fetchInfo2); DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( - remoteFetchTask, - future, - fetchInfo, + remoteFetchTasks, + remoteFetchResults, + remoteFetchInfos, remoteFetchMaxWaitMs, - Map.of(topicIdPartition, List.of(fetchStatus)), + Map.of(topicIdPartition, fetchStatus1, topicIdPartition2, fetchStatus2), fetchParams, - Map.of(topicIdPartition, List.of(logReadInfo)), + Map.of(topicIdPartition, logReadInfo1, topicIdPartition2, logReadInfo2), partitionOrException, callback ); + // Verify that the ExpiresPerSec metric is zero before fetching + long existingMetricVal = expiresPerSecValue(); + // Verify the delayedRemoteFetch is not completed yet + assertFalse(delayedRemoteFetch.isCompleted()); + // Force the delayed remote fetch to expire delayedRemoteFetch.run(); - // Check that the task was cancelled and force-completed - verify(remoteFetchTask).cancel(false); + // Check that the expired task was cancelled and force-completed + verify(remoteFetchTaskExpired).cancel(anyBoolean()); + verify(remoteFetchTask2, never()).cancel(anyBoolean()); assertTrue(delayedRemoteFetch.isCompleted()); - long metricsCount = KafkaYammerMetrics.defaultRegistry().allMetrics().keySet().stream() - .filter(m -> m.getMBeanName().equals("kafka.server:type=DelayedRemoteFetchMetrics,name=ExpiresPerSec")) - .count(); - assertEquals(1, metricsCount); + // Check that the ExpiresPerSec metric was incremented + assertTrue(expiresPerSecValue() > existingMetricVal); - // Fetch results should still include local read results - assertNotNull(actualTopicPartition.get()); - assertEquals(topicIdPartition, actualTopicPartition.get()); - assertNotNull(fetchResultOpt.get()); + // Fetch results should include 2 results and the expired one should return local read results + assertEquals(2, responses.size()); + assertTrue(responses.containsKey(topicIdPartition)); + assertTrue(responses.containsKey(topicIdPartition2)); - FetchPartitionData fetchResult = fetchResultOpt.get(); - assertEquals(Errors.NONE, fetchResult.error); - assertEquals(highWatermark, fetchResult.highWatermark); - assertEquals(leaderLogStartOffset, fetchResult.logStartOffset); + assertEquals(Errors.NONE, responses.get(topicIdPartition).error); + assertEquals(highWatermark, responses.get(topicIdPartition).highWatermark); + assertEquals(leaderLogStartOffset, responses.get(topicIdPartition).logStartOffset); + + assertEquals(Errors.NONE, responses.get(topicIdPartition2).error); + } + + @Test + public void testMultiplePartitions() { + Map responses = new HashMap<>(); + + Consumer> callback = responses::putAll; + + // Create futures - one completed, one not + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + // Only complete one remote fetch + future1.complete(buildRemoteReadResult(Errors.NONE)); + + RemoteStorageFetchInfo fetchInfo1 = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + RemoteStorageFetchInfo fetchInfo2 = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + + long highWatermark1 = 100L; + long leaderLogStartOffset1 = 10L; + long highWatermark2 = 200L; + long leaderLogStartOffset2 = 20L; + + LogReadResult logReadInfo1 = buildReadResult(Errors.NONE, highWatermark1, leaderLogStartOffset1); + LogReadResult logReadInfo2 = buildReadResult(Errors.NONE, highWatermark2, leaderLogStartOffset2); + + FetchPartitionStatus fetchStatus1 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)); + + FetchPartitionStatus fetchStatus2 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset + 100L), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset + 100L, logStartOffset, maxBytes, currentLeaderEpoch)); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + Map.of(), + Map.of(topicIdPartition, future1, topicIdPartition2, future2), + Map.of(topicIdPartition, fetchInfo1, topicIdPartition2, fetchInfo2), + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, fetchStatus1, topicIdPartition2, fetchStatus2), + fetchParams, + Map.of(topicIdPartition, logReadInfo1, topicIdPartition2, logReadInfo2), + partitionOrException, + callback + ); + + // Should not complete since future2 is not done + assertFalse(delayedRemoteFetch.tryComplete()); + assertFalse(delayedRemoteFetch.isCompleted()); + + // Complete future2 + future2.complete(buildRemoteReadResult(Errors.NONE)); + + // Now it should complete + assertTrue(delayedRemoteFetch.tryComplete()); + assertTrue(delayedRemoteFetch.isCompleted()); + + // Verify both partitions were processed without error + assertEquals(2, responses.size()); + assertTrue(responses.containsKey(topicIdPartition)); + assertTrue(responses.containsKey(topicIdPartition2)); + + assertEquals(Errors.NONE, responses.get(topicIdPartition).error); + assertEquals(highWatermark1, responses.get(topicIdPartition).highWatermark); + assertEquals(leaderLogStartOffset1, responses.get(topicIdPartition).logStartOffset); + + assertEquals(Errors.NONE, responses.get(topicIdPartition2).error); + assertEquals(highWatermark2, responses.get(topicIdPartition2).highWatermark); + assertEquals(leaderLogStartOffset2, responses.get(topicIdPartition2).logStartOffset); + } + + @Test + public void testMultiplePartitionsWithFailedResults() { + Map responses = new HashMap<>(); + + Consumer> callback = responses::putAll; + + // Create futures - one successful, one with error + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + + // Created 1 successful result and 1 failed result + future1.complete(buildRemoteReadResult(Errors.NONE)); + future2.complete(buildRemoteReadResult(Errors.UNKNOWN_SERVER_ERROR)); + + RemoteStorageFetchInfo fetchInfo1 = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + RemoteStorageFetchInfo fetchInfo2 = new RemoteStorageFetchInfo(0, false, topicIdPartition, null, null); + + LogReadResult logReadInfo1 = buildReadResult(Errors.NONE, 100, 10); + LogReadResult logReadInfo2 = buildReadResult(Errors.NONE, 100, 10); + + FetchPartitionStatus fetchStatus1 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset, logStartOffset, maxBytes, currentLeaderEpoch)); + + FetchPartitionStatus fetchStatus2 = new FetchPartitionStatus( + new LogOffsetMetadata(fetchOffset + 100), + new FetchRequest.PartitionData(Uuid.ZERO_UUID, fetchOffset + 100, logStartOffset, maxBytes, currentLeaderEpoch)); + + DelayedRemoteFetch delayedRemoteFetch = new DelayedRemoteFetch( + Map.of(), + Map.of(topicIdPartition, future1, topicIdPartition2, future2), + Map.of(topicIdPartition, fetchInfo1, topicIdPartition2, fetchInfo2), + remoteFetchMaxWaitMs, + Map.of(topicIdPartition, fetchStatus1, topicIdPartition2, fetchStatus2), + fetchParams, + Map.of(topicIdPartition, logReadInfo1, topicIdPartition2, logReadInfo2), + partitionOrException, + callback + ); + + assertTrue(delayedRemoteFetch.tryComplete()); + assertTrue(delayedRemoteFetch.isCompleted()); + + // Verify both partitions were processed + assertEquals(2, responses.size()); + assertTrue(responses.containsKey(topicIdPartition)); + assertTrue(responses.containsKey(topicIdPartition2)); + + // First partition should be successful + FetchPartitionData fetchResult1 = responses.get(topicIdPartition); + assertEquals(Errors.NONE, fetchResult1.error); + + // Second partition should have an error due to remote fetch failure + FetchPartitionData fetchResult2 = responses.get(topicIdPartition2); + assertEquals(Errors.UNKNOWN_SERVER_ERROR, fetchResult2.error); } private FetchParams buildFetchParams(int replicaId, int maxWaitMs) { @@ -298,9 +458,10 @@ private LogReadResult buildReadResult(Errors error) { return buildReadResult(error, 0, 0); } - private LogReadResult buildReadResult(Errors error, int highWatermark, int leaderLogStartOffset) { + private LogReadResult buildReadResult(Errors error, long highWatermark, long leaderLogStartOffset) { return new LogReadResult( - new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), + new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY, false, Optional.empty(), + Optional.of(mock(RemoteStorageFetchInfo.class))), Optional.empty(), highWatermark, leaderLogStartOffset, @@ -310,4 +471,10 @@ private LogReadResult buildReadResult(Errors error, int highWatermark, int leade OptionalLong.empty(), error != Errors.NONE ? Optional.of(error.exception()) : Optional.empty()); } + + private RemoteLogReadResult buildRemoteReadResult(Errors error) { + return new RemoteLogReadResult( + Optional.of(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY)), + error != Errors.NONE ? Optional.of(error.exception()) : Optional.empty()); + } } \ No newline at end of file From 20ec186d22ac3a68aa8bc3e9c76929aaef9e5f5c Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 22 Jul 2025 12:02:58 +0800 Subject: [PATCH 06/17] delete empty file --- core/src/main/scala/kafka/server/DelayedRemoteFetch.scala | 0 .../scala/integration/kafka/server/DelayedRemoteFetchTest.scala | 0 2 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/DelayedRemoteFetch.scala delete mode 100644 core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedRemoteFetchTest.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 From 67c3c18a09666214b1e73c97ac8d5bfec265ddde Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 5 Aug 2025 09:07:33 +0800 Subject: [PATCH 07/17] address comment --- .../apache/kafka/server/purgatory/DelayedRemoteFetch.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 32e27f78fc867..d49a1349c330e 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -55,6 +55,7 @@ public class DelayedRemoteFetch extends DelayedOperation { private static final Logger LOG = LoggerFactory.getLogger(DelayedRemoteFetch.class); + // For compatibility, metrics are defined to be under `kafka.server.DelayedRemoteFetchMetrics` class private static final KafkaMetricsGroup METRICS_GROUP = new KafkaMetricsGroup("kafka.server", "DelayedRemoteFetchMetrics"); static final Meter EXPIRED_REQUEST_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); @@ -162,11 +163,11 @@ public void onComplete() { && remoteFetchResult.isDone() && result.error() == Errors.NONE && result.info().delayedRemoteStorageFetch.isPresent()) { - if (remoteFetchResult.get().error.isPresent()) { + if (remoteFetchResult.get().error().isPresent()) { fetchPartitionData.put(tp, - new LogReadResult(remoteFetchResult.get().error.get()).toFetchPartitionData(false)); + new LogReadResult(remoteFetchResult.get().error().get()).toFetchPartitionData(false)); } else { - FetchDataInfo info = remoteFetchResult.get().fetchDataInfo.get(); + FetchDataInfo info = remoteFetchResult.get().fetchDataInfo().get(); fetchPartitionData.put(tp, new FetchPartitionData( result.error(), From 6aa0df8f18df7211cb7c52c1385a42ab4400a33f Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Fri, 26 Sep 2025 10:05:55 +0800 Subject: [PATCH 08/17] address comment --- .../org/apache/kafka/server/FetchPartitionStatus.java | 2 +- .../kafka/server/purgatory/DelayedRemoteFetch.java | 11 +++++++---- .../server/purgatory/DelayedRemoteFetchTest.java | 2 +- 3 files changed, 9 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java b/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java index 06dcfa7b840ea..9e0acfbf1dc16 100644 --- a/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java +++ b/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java @@ -31,7 +31,7 @@ public record FetchPartitionStatus( @Override public String toString() { return String.format( - "[startOffsetMetadata: %s, fetchInfo: %s", + "[startOffsetMetadata: %s, fetchInfo: %s]", startOffsetMetadata, fetchInfo ); diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index d49a1349c330e..75255fe5c5812 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -58,7 +58,7 @@ public class DelayedRemoteFetch extends DelayedOperation { // For compatibility, metrics are defined to be under `kafka.server.DelayedRemoteFetchMetrics` class private static final KafkaMetricsGroup METRICS_GROUP = new KafkaMetricsGroup("kafka.server", "DelayedRemoteFetchMetrics"); - static final Meter EXPIRED_REQUEST_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); + private static final Meter EXPIRED_REQUEST_METER = METRICS_GROUP.newMeter("ExpiresPerSec", "requests", TimeUnit.SECONDS); private final Map> remoteFetchTasks; private final Map> remoteFetchResults; @@ -95,10 +95,13 @@ public DelayedRemoteFetch(Map> remoteFetchTasks, /** * 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: All the remote storage read request completed (succeeded or failed) + *

+ * Case c: All the remote storage read requests 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 @@ -195,4 +198,4 @@ public void onComplete() { public static Meter expiredRequestMeter() { return EXPIRED_REQUEST_METER; } -} \ No newline at end of file +} diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index 878c1ed37a95b..a5cc06c6e391a 100644 --- a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -477,4 +477,4 @@ private RemoteLogReadResult buildRemoteReadResult(Errors error) { Optional.of(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY)), error != Errors.NONE ? Optional.of(error.exception()) : Optional.empty()); } -} \ No newline at end of file +} From 544e04fcefe3afa2b19d2025ca6974137a611d07 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Mon, 29 Sep 2025 18:35:04 +0800 Subject: [PATCH 09/17] address comment --- .../org/apache/kafka/server/purgatory/DelayedRemoteFetch.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 75255fe5c5812..5467cf630e421 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -163,7 +163,8 @@ public void onComplete() { CompletableFuture remoteFetchResult = remoteFetchResults.get(tp); if (remoteFetchInfos.containsKey(tp) - && remoteFetchResult.isDone() && result.error() == Errors.NONE + && remoteFetchResult.isDone() + && result.error() == Errors.NONE && result.info().delayedRemoteStorageFetch.isPresent()) { if (remoteFetchResult.get().error().isPresent()) { From 941b2d39557417a2f2cf6906d40a5094ab882e02 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 30 Sep 2025 09:47:44 +0800 Subject: [PATCH 10/17] move DelayedRemoteFetch to storage module --- core/src/main/java/kafka/server/share/DelayedShareFetch.java | 2 +- .../main/java/kafka/server/share/PendingRemoteFetches.java | 2 +- core/src/main/scala/kafka/server/DelayedFetch.scala | 3 +-- core/src/main/scala/kafka/server/ReplicaManager.scala | 4 ++-- .../test/java/kafka/server/share/DelayedShareFetchTest.java | 2 +- .../java/kafka/server/share/SharePartitionManagerTest.java | 2 +- .../scala/integration/kafka/server/DelayedFetchTest.scala | 3 +-- .../scala/unit/kafka/server/ReplicaManagerQuotasTest.scala | 3 +-- .../src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 4 ++-- .../apache/kafka/server/purgatory/DelayedRemoteFetchTest.java | 4 ++-- .../org/apache/kafka/server/purgatory/DelayedRemoteFetch.java | 4 ++-- .../kafka/storage/internals/log}/FetchPartitionStatus.java | 2 +- .../apache/kafka/storage/internals/log}/LogReadResult.java | 2 +- 13 files changed, 17 insertions(+), 20 deletions(-) rename {server => storage}/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java (98%) rename {server/src/main/java/org/apache/kafka/server => storage/src/main/java/org/apache/kafka/storage/internals/log}/FetchPartitionStatus.java (96%) rename {server/src/main/java/org/apache/kafka/server => storage/src/main/java/org/apache/kafka/storage/internals/log}/LogReadResult.java (99%) diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index c7e6c7d449496..ac00eb22cdf72 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -30,7 +30,7 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.purgatory.DelayedOperation; import org.apache.kafka.server.share.SharePartitionKey; diff --git a/core/src/main/java/kafka/server/share/PendingRemoteFetches.java b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java index 575a32ef4662e..829e2d78c0347 100644 --- a/core/src/main/java/kafka/server/share/PendingRemoteFetches.java +++ b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java @@ -17,7 +17,7 @@ package kafka.server.share; import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index f0d0cb22fc904..0a634d06ca519 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -25,11 +25,10 @@ import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.errors._ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} -import org.apache.kafka.server.FetchPartitionStatus import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.purgatory.DelayedOperation import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} -import org.apache.kafka.storage.internals.log.LogOffsetMetadata +import org.apache.kafka.storage.internals.log.{FetchPartitionStatus, LogOffsetMetadata} import scala.collection._ import scala.jdk.CollectionConverters._ diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 3f876b40c51af..b00ceffb9dae9 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -63,9 +63,9 @@ import org.apache.kafka.server.transaction.AddPartitionsToTxnManager import org.apache.kafka.server.transaction.AddPartitionsToTxnManager.TransactionSupportedOperation import org.apache.kafka.server.util.timer.{SystemTimer, TimerTask} import org.apache.kafka.server.util.{Scheduler, ShutdownableThread} -import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, FetchPartitionStatus, LogReadResult, common} +import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, common} import org.apache.kafka.storage.internals.checkpoint.{LazyOffsetCheckpoints, OffsetCheckpointFile, OffsetCheckpoints} -import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogReadInfo, OffsetResultHolder, RecordValidationException, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, FetchPartitionStatus, LeaderHwChange, LogAppendInfo, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogReadInfo, LogReadResult, OffsetResultHolder, RecordValidationException, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import java.io.File diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index ffa9f8b11456a..c2d010cd414cf 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -32,7 +32,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.log.remote.storage.RemoteLogManager; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 24a84bab64a9b..407ae14e62331 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -51,7 +51,7 @@ import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.GroupConfigManager; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.common.ShareVersion; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala index 4f1acf60130e5..c328f6f20bfa2 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala @@ -25,9 +25,8 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEnd import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.FetchRequest -import org.apache.kafka.server.{FetchPartitionStatus, LogReadResult} import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams, FetchPartitionData} -import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogOffsetMetadata, LogOffsetSnapshot} +import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchPartitionStatus, LogOffsetMetadata, LogOffsetSnapshot, LogReadResult} import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala index 496f1b7eaa0bf..307afad4f5f45 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala @@ -30,11 +30,10 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState -import org.apache.kafka.server.FetchPartitionStatus import org.apache.kafka.server.common.KRaftVersion import org.apache.kafka.server.storage.log.{FetchIsolation, FetchParams} import org.apache.kafka.server.util.{KafkaScheduler, MockTime} -import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogOffsetSnapshot, UnifiedLog} +import org.apache.kafka.storage.internals.log.{FetchDataInfo, FetchPartitionStatus, LogConfig, LogDirFailureChannel, LogOffsetMetadata, LogOffsetSnapshot, UnifiedLog} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyLong} diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 985b8ffef5d0d..efe0d77c7855a 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -63,7 +63,7 @@ import org.apache.kafka.server.log.remote.TopicPartitionLog import org.apache.kafka.server.log.remote.storage._ import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics} import org.apache.kafka.server.network.BrokerEndPoint -import org.apache.kafka.server.{LogReadResult, PartitionFetchState} +import org.apache.kafka.server.PartitionFetchState import org.apache.kafka.server.purgatory.{DelayedDeleteRecords, DelayedOperationPurgatory, DelayedRemoteFetch, DelayedRemoteListOffsets} import org.apache.kafka.server.share.SharePartitionKey import org.apache.kafka.server.share.fetch.{DelayedShareFetchGroupKey, DelayedShareFetchKey, ShareFetch} @@ -76,7 +76,7 @@ import org.apache.kafka.server.util.timer.MockTimer import org.apache.kafka.server.util.{MockScheduler, MockTime, Scheduler} import org.apache.kafka.storage.internals.checkpoint.LazyOffsetCheckpoints import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache -import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, FetchDataInfo, LocalLog, LogAppendInfo, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetsListener, LogOffsetSnapshot, LogSegments, ProducerStateManager, ProducerStateManagerConfig, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard} +import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, FetchDataInfo, LocalLog, LogAppendInfo, LogConfig, LogDirFailureChannel, LogLoader, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogReadResult, LogSegments, ProducerStateManager, ProducerStateManagerConfig, RemoteLogReadResult, RemoteStorageFetchInfo, UnifiedLog, VerificationGuard} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterAll, AfterEach, BeforeEach, Test} diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index a5cc06c6e391a..00de3bb11b8a7 100644 --- a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchRequest; -import org.apache.kafka.server.FetchPartitionStatus; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.FetchPartitionStatus; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaYammerMetrics; import org.apache.kafka.server.storage.log.FetchIsolation; import org.apache.kafka.server.storage.log.FetchParams; diff --git a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java similarity index 98% rename from server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java rename to storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 5467cf630e421..9e9f8923dc2f8 100644 --- a/server/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.server.FetchPartitionStatus; -import org.apache.kafka.server.LogReadResult; +import org.apache.kafka.storage.internals.log.FetchPartitionStatus; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.storage.log.FetchParams; import org.apache.kafka.server.storage.log.FetchPartitionData; diff --git a/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java similarity index 96% rename from server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java rename to storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java index 9e0acfbf1dc16..1229363376224 100644 --- a/server/src/main/java/org/apache/kafka/server/FetchPartitionStatus.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.server; +package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.requests.FetchRequest.PartitionData; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; diff --git a/server/src/main/java/org/apache/kafka/server/LogReadResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java similarity index 99% rename from server/src/main/java/org/apache/kafka/server/LogReadResult.java rename to storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java index 9a39eca8e202b..82f8421b1ea9a 100644 --- a/server/src/main/java/org/apache/kafka/server/LogReadResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.server; +package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.protocol.Errors; From d222a5a27ef8f1e2161666bc9c48e6215b7f4446 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 30 Sep 2025 09:54:31 +0800 Subject: [PATCH 11/17] fix ci --- core/src/main/java/kafka/server/share/DelayedShareFetch.java | 2 +- core/src/main/java/kafka/server/share/PendingRemoteFetches.java | 2 +- .../src/test/java/kafka/server/share/DelayedShareFetchTest.java | 2 +- .../test/java/kafka/server/share/SharePartitionManagerTest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index ac00eb22cdf72..251e597d53a0a 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -30,7 +30,6 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.errors.NotLeaderException; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.purgatory.DelayedOperation; import org.apache.kafka.server.share.SharePartitionKey; @@ -46,6 +45,7 @@ import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; diff --git a/core/src/main/java/kafka/server/share/PendingRemoteFetches.java b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java index 829e2d78c0347..c25cc70677519 100644 --- a/core/src/main/java/kafka/server/share/PendingRemoteFetches.java +++ b/core/src/main/java/kafka/server/share/PendingRemoteFetches.java @@ -17,8 +17,8 @@ package kafka.server.share; import org.apache.kafka.common.TopicIdPartition; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index c2d010cd414cf..0741e52e3dd02 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -32,7 +32,6 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.log.remote.storage.RemoteLogManager; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; @@ -52,6 +51,7 @@ import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 407ae14e62331..d78bae698b5fe 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -51,7 +51,6 @@ import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.GroupConfigManager; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.common.ShareVersion; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; @@ -84,6 +83,7 @@ import org.apache.kafka.server.util.timer.TimerTask; import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.OffsetResultHolder; import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; From ea5539daa532ec1272e9fb172b1117095ba16867 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 30 Sep 2025 10:40:47 +0800 Subject: [PATCH 12/17] fix ci --- checkstyle/import-control-storage.xml | 5 ++++- .../kafka/server/purgatory/DelayedRemoteFetchTest.java | 4 ++-- .../apache/kafka/server/purgatory/DelayedRemoteFetch.java | 4 ++-- .../kafka/storage/internals/log/FetchPartitionStatus.java | 1 - .../apache/kafka/storage/internals/log/LogReadResult.java | 3 --- 5 files changed, 8 insertions(+), 9 deletions(-) diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index 2a0f74126859a..e4194b20e2c99 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -84,6 +84,10 @@ + + + + @@ -164,7 +168,6 @@ - diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index 00de3bb11b8a7..7620f36aebd53 100644 --- a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -23,14 +23,14 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.requests.FetchRequest; -import org.apache.kafka.storage.internals.log.FetchPartitionStatus; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaYammerMetrics; import org.apache.kafka.server.storage.log.FetchIsolation; import org.apache.kafka.server.storage.log.FetchParams; import org.apache.kafka.server.storage.log.FetchPartitionData; import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.FetchPartitionStatus; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; diff --git a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 9e9f8923dc2f8..2a9965394530d 100644 --- a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -23,13 +23,13 @@ import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.protocol.Errors; -import org.apache.kafka.storage.internals.log.FetchPartitionStatus; -import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.storage.log.FetchParams; import org.apache.kafka.server.storage.log.FetchPartitionData; import org.apache.kafka.storage.internals.log.FetchDataInfo; +import org.apache.kafka.storage.internals.log.FetchPartitionStatus; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; +import org.apache.kafka.storage.internals.log.LogReadResult; import org.apache.kafka.storage.internals.log.RemoteLogReadResult; import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java index 1229363376224..16cc1991d49ec 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java @@ -18,7 +18,6 @@ package org.apache.kafka.storage.internals.log; import org.apache.kafka.common.requests.FetchRequest.PartitionData; -import org.apache.kafka.storage.internals.log.LogOffsetMetadata; /** * A class containing log offset metadata and fetch info for a topic partition. diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java index 82f8421b1ea9a..f5eecb60b420f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java @@ -20,9 +20,6 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.server.storage.log.FetchPartitionData; -import org.apache.kafka.storage.internals.log.FetchDataInfo; -import org.apache.kafka.storage.internals.log.LogOffsetMetadata; -import org.apache.kafka.storage.internals.log.UnifiedLog; import java.util.Optional; import java.util.OptionalInt; From 11f7708d31ccc7d71751eb2a1c4dc158aa574097 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 14 Oct 2025 09:19:01 +0800 Subject: [PATCH 13/17] fix conflict --- core/src/main/scala/kafka/server/DelayedRemoteFetch.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 core/src/main/scala/kafka/server/DelayedRemoteFetch.scala diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 From fd4e3f9d21dfdd649c700e0c6655f84cac5428b6 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Tue, 21 Oct 2025 10:00:39 +0800 Subject: [PATCH 14/17] fix comment --- .../internals/log/FetchPartitionStatus.java | 2 +- .../storage/internals/log/LogReadResult.java | 27 ------------------- .../purgatory/DelayedRemoteFetchTest.java | 0 3 files changed, 1 insertion(+), 28 deletions(-) rename {server => storage}/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java (100%) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java index 16cc1991d49ec..34edb53e2f52f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java @@ -30,7 +30,7 @@ public record FetchPartitionStatus( @Override public String toString() { return String.format( - "[startOffsetMetadata: %s, fetchInfo: %s]", + "FetchPartitionStatus(startOffsetMetadata: %s, fetchInfo: %s)", startOffsetMetadata, fetchInfo ); diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java index f5eecb60b420f..fe3ae9453c41c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java @@ -52,19 +52,6 @@ public record LogReadResult( OptionalInt preferredReadReplica, Optional exception ) { - public LogReadResult( - FetchDataInfo info, - Optional divergingEpoch, - long highWatermark, - long leaderLogStartOffset, - long leaderLogEndOffset, - long followerLogStartOffset, - long fetchTimeMs, - OptionalLong lastStableOffset) { - this(info, divergingEpoch, highWatermark, leaderLogStartOffset, leaderLogEndOffset, followerLogStartOffset, - fetchTimeMs, lastStableOffset, OptionalInt.empty(), Optional.empty()); - } - public LogReadResult( FetchDataInfo info, Optional divergingEpoch, @@ -79,20 +66,6 @@ public LogReadResult( fetchTimeMs, lastStableOffset, OptionalInt.empty(), exception); } - public LogReadResult( - FetchDataInfo info, - Optional divergingEpoch, - long highWatermark, - long leaderLogStartOffset, - long leaderLogEndOffset, - long followerLogStartOffset, - long fetchTimeMs, - OptionalLong lastStableOffset, - OptionalInt preferredReadReplica) { - this(info, divergingEpoch, highWatermark, leaderLogStartOffset, leaderLogEndOffset, followerLogStartOffset, - fetchTimeMs, lastStableOffset, preferredReadReplica, Optional.empty()); - } - public LogReadResult(Throwable e) { this(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), Optional.empty(), diff --git a/server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java similarity index 100% rename from server/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java rename to storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java From cdbdc007340cc7c92d1f171235a8dd325236df96 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 22 Oct 2025 16:00:27 +0800 Subject: [PATCH 15/17] fix comment --- .../kafka/server/share/DelayedShareFetch.java | 3 +- .../scala/kafka/server/ReplicaManager.scala | 14 ++-- .../server/share/DelayedShareFetchTest.java | 8 +- .../share/SharePartitionManagerTest.java | 2 +- .../kafka/server/DelayedFetchTest.scala | 2 +- .../kafka/server/ReplicaManagerTest.scala | 4 +- .../server/purgatory/DelayedRemoteFetch.java | 76 ++++++++----------- .../internals/log/FetchPartitionStatus.java | 9 --- .../storage/internals/log/LogReadResult.java | 33 ++------ .../purgatory/DelayedRemoteFetchTest.java | 2 +- 10 files changed, 56 insertions(+), 97 deletions(-) diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index 251e597d53a0a..7b061a28bd50e 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -852,13 +852,12 @@ private void completeRemoteStorageShareFetchRequest() { if (remoteFetch.remoteFetchResult().isDone()) { RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); if (remoteLogReadResult.error().isPresent()) { - Throwable error = remoteLogReadResult.error().get(); // If there is any error for the remote fetch topic partition, we populate the error accordingly. shareFetchPartitionData.add( new ShareFetchPartitionData( remoteFetch.topicIdPartition(), partitionsAcquired.get(remoteFetch.topicIdPartition()), - new LogReadResult(error).toFetchPartitionData(false) + new LogReadResult(Errors.forException(remoteLogReadResult.error().get())).toFetchPartitionData(false) ) ); } else { diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 365f46f14ed2f..54cf63d5e02b7 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -188,7 +188,7 @@ object ReplicaManager { -1L, -1L, OptionalLong.empty(), - Optional.of(e)) + Errors.forException(e)); } private[server] def isListOffsetsTimestampUnsupported(timestamp: JLong, version: Short): Boolean = { @@ -1807,7 +1807,7 @@ class ReplicaManager(val config: KafkaConfig, -1L, OptionalLong.of(offsetSnapshot.lastStableOffset.messageOffset), if (preferredReadReplica.isDefined) OptionalInt.of(preferredReadReplica.get) else OptionalInt.empty(), - Optional.empty()) + Errors.NONE) } else { log = partition.localLogWithEpochOrThrow(fetchInfo.currentLeaderEpoch, params.fetchOnlyLeader()) @@ -1831,7 +1831,7 @@ class ReplicaManager(val config: KafkaConfig, fetchTimeMs, OptionalLong.of(readInfo.lastStableOffset), if (preferredReadReplica.isDefined) OptionalInt.of(preferredReadReplica.get) else OptionalInt.empty(), - Optional.empty() + Errors.NONE ) } } catch { @@ -1844,7 +1844,7 @@ class ReplicaManager(val config: KafkaConfig, _: ReplicaNotAvailableException | _: KafkaStorageException | _: InconsistentTopicIdException) => - new LogReadResult(e) + new LogReadResult(Errors.forException(e)) case e: OffsetOutOfRangeException => handleOffsetOutOfRangeError(tp, params, fetchInfo, adjustedMaxBytes, minOneMessage, log, fetchTimeMs, e) case e: Throwable => @@ -1863,7 +1863,7 @@ class ReplicaManager(val config: KafkaConfig, UnifiedLog.UNKNOWN_OFFSET, -1L, OptionalLong.empty(), - Optional.of(e) + Errors.forException(e) ) } } @@ -1944,10 +1944,10 @@ class ReplicaManager(val config: KafkaConfig, fetchInfo.logStartOffset, fetchTimeMs, OptionalLong.of(log.lastStableOffset), - Optional.empty[Throwable]()) + Errors.NONE) } } else { - new LogReadResult(exception) + new LogReadResult(Errors.forException(exception)) } } diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index 0741e52e3dd02..8aab8eb5495c7 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -2023,7 +2023,7 @@ public void testRemoteStorageFetchCompletionPostRegisteringCallbackByPendingFetc -1L, OptionalLong.empty(), OptionalInt.empty(), - Optional.empty() + Errors.NONE )); when(pendingRemoteFetches.remoteFetches()).thenReturn(List.of(remoteFetch)); when(pendingRemoteFetches.isDone()).thenReturn(false); @@ -2104,7 +2104,7 @@ public void testRemoteStorageFetchCompletionPostRegisteringCallbackByTimerTaskCo -1L, OptionalLong.empty(), OptionalInt.empty(), - Optional.empty() + Errors.NONE )); when(pendingRemoteFetches.remoteFetches()).thenReturn(List.of(remoteFetch)); when(pendingRemoteFetches.isDone()).thenReturn(false); @@ -2179,7 +2179,7 @@ private Seq> buildLocalAndRemoteFetchRes -1L, OptionalLong.empty(), OptionalInt.empty(), - Optional.empty() + Errors.NONE )))); remoteReadTopicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( REMOTE_FETCH_INFO, @@ -2191,7 +2191,7 @@ private Seq> buildLocalAndRemoteFetchRes -1L, OptionalLong.empty(), OptionalInt.empty(), - Optional.empty() + Errors.NONE )))); return CollectionConverters.asScala(logReadResults).toSeq(); } diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index d78bae698b5fe..541719aea0678 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -3226,7 +3226,7 @@ Compression.NONE, new SimpleRecord("test-key".getBytes(), "test-value".getBytes( -1L, OptionalLong.empty(), OptionalInt.empty(), - Optional.empty() + Errors.NONE )))); return CollectionConverters.asScala(logReadResults).toSeq(); } diff --git a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala index c328f6f20bfa2..fa3b8465d651f 100644 --- a/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala +++ b/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala @@ -264,7 +264,7 @@ class DelayedFetchTest { -1L, -1L, OptionalLong.empty(), - if (error != Errors.NONE) Optional.of[Throwable](error.exception) else Optional.empty[Throwable]()) + error) } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index d13dc7064cc35..339bcf0bb7ae7 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -3575,13 +3575,13 @@ class ReplicaManagerTest { mock(classOf[FetchDataInfo]) }).when(spyRLM).read(any()) - val curExpiresPerSec = DelayedRemoteFetch.expiredRequestMeter().count() + val curExpiresPerSec = DelayedRemoteFetch.expiredRequestCount() replicaManager.fetchMessages(params, Seq(tidp0 -> new PartitionData(topicId, fetchOffset, 0, 100000, Optional.of[Integer](leaderEpoch), Optional.of[Integer](leaderEpoch))), UNBOUNDED_QUOTA, fetchCallback) // advancing the clock to expire the delayed remote fetch timer.advanceClock(2000L) // verify the DelayedRemoteFetchMetrics.expiredRequestMeter.mark is called since the delayed remote fetch is expired - TestUtils.waitUntilTrue(() => (curExpiresPerSec + 1) == DelayedRemoteFetch.expiredRequestMeter().count(), "DelayedRemoteFetchMetrics.expiredRequestMeter.count() should be 1, but got: " + DelayedRemoteFetch.expiredRequestMeter().count(), 10000L) + TestUtils.waitUntilTrue(() => (curExpiresPerSec + 1) == DelayedRemoteFetch.expiredRequestCount(), "DelayedRemoteFetchMetrics.expiredRequestMeter.count() should be 1, but got: " + DelayedRemoteFetch.expiredRequestCount(), 10000L) latch.countDown() } finally { Utils.tryAll(util.Arrays.asList[Callable[Void]]( diff --git a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 2a9965394530d..517f4390a974e 100644 --- a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -38,11 +38,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -139,13 +138,11 @@ public boolean tryComplete() { public void onExpiration() { // cancel the remote storage read task, if it has not been executed yet and // avoid interrupting the task if it is already running as it may force closing opened/cached resources as transaction index. - remoteFetchTasks.forEach(((topicIdPartition, task) -> { - if (task != null && !task.isDone()) { - if (!task.cancel(false)) { - LOG.debug("Remote fetch task for remoteFetchInfo: {} could not be cancelled.", remoteFetchInfos.get(topicIdPartition)); - } + remoteFetchTasks.forEach((topicIdPartition, task) -> { + if (task != null && !task.isDone() && !task.cancel(false)) { + LOG.debug("Remote fetch task for remoteFetchInfo: {} could not be cancelled.", remoteFetchInfos.get(topicIdPartition)); } - })); + }); EXPIRED_REQUEST_METER.mark(); } @@ -155,48 +152,41 @@ public void onExpiration() { */ @Override public void onComplete() { - Map fetchPartitionData = new HashMap<>(); - try { - for (Map.Entry entry : localReadResults.entrySet()) { - TopicIdPartition tp = entry.getKey(); - LogReadResult result = entry.getValue(); - - CompletableFuture remoteFetchResult = remoteFetchResults.get(tp); - if (remoteFetchInfos.containsKey(tp) - && remoteFetchResult.isDone() - && result.error() == Errors.NONE - && result.info().delayedRemoteStorageFetch.isPresent()) { - - if (remoteFetchResult.get().error().isPresent()) { - fetchPartitionData.put(tp, - new LogReadResult(remoteFetchResult.get().error().get()).toFetchPartitionData(false)); - } else { - FetchDataInfo info = remoteFetchResult.get().fetchDataInfo().get(); - fetchPartitionData.put(tp, - new FetchPartitionData( - result.error(), - result.highWatermark(), - result.leaderLogStartOffset(), - info.records, - Optional.empty(), - result.lastStableOffset(), - info.abortedTransactions, - result.preferredReadReplica(), - false)); - } + Map fetchPartitionData = new LinkedHashMap<>(); + localReadResults.forEach((tpId, result) -> { + CompletableFuture remoteFetchResult = remoteFetchResults.get(tpId); + if (remoteFetchInfos.containsKey(tpId) + && remoteFetchResult.isDone() + && result.error() == Errors.NONE + && result.info().delayedRemoteStorageFetch.isPresent()) { + + if (remoteFetchResult.join().error().isPresent()) { + fetchPartitionData.put(tpId, + new LogReadResult(Errors.forException(remoteFetchResult.join().error().get())).toFetchPartitionData(false)); } else { - fetchPartitionData.put(tp, result.toFetchPartitionData(false)); + FetchDataInfo info = remoteFetchResult.join().fetchDataInfo().get(); + fetchPartitionData.put(tpId, + new FetchPartitionData( + result.error(), + result.highWatermark(), + result.leaderLogStartOffset(), + info.records, + Optional.empty(), + result.lastStableOffset(), + info.abortedTransactions, + result.preferredReadReplica(), + false)); } + } else { + fetchPartitionData.put(tpId, result.toFetchPartitionData(false)); } - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } + }); responseCallback.accept(fetchPartitionData); } // Visible for testing - public static Meter expiredRequestMeter() { - return EXPIRED_REQUEST_METER; + public static long expiredRequestCount() { + return EXPIRED_REQUEST_METER.count(); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java index 34edb53e2f52f..5a060013b5e75 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/FetchPartitionStatus.java @@ -26,13 +26,4 @@ public record FetchPartitionStatus( LogOffsetMetadata startOffsetMetadata, PartitionData fetchInfo ) { - - @Override - public String toString() { - return String.format( - "FetchPartitionStatus(startOffsetMetadata: %s, fetchInfo: %s)", - startOffsetMetadata, - fetchInfo - ); - } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java index fe3ae9453c41c..057a66a25c10a 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogReadResult.java @@ -38,7 +38,7 @@ * @param fetchTimeMs The time the fetch was received * @param lastStableOffset Current LSO or None if the result has an exception * @param preferredReadReplica the preferred read replica to be used for future fetches - * @param exception Exception if error encountered while reading from the log + * @param error Errors if error encountered while reading from the log */ public record LogReadResult( FetchDataInfo info, @@ -50,7 +50,7 @@ public record LogReadResult( long fetchTimeMs, OptionalLong lastStableOffset, OptionalInt preferredReadReplica, - Optional exception + Errors error ) { public LogReadResult( FetchDataInfo info, @@ -61,12 +61,12 @@ public LogReadResult( long followerLogStartOffset, long fetchTimeMs, OptionalLong lastStableOffset, - Optional exception) { + Errors error) { this(info, divergingEpoch, highWatermark, leaderLogStartOffset, leaderLogEndOffset, followerLogStartOffset, - fetchTimeMs, lastStableOffset, OptionalInt.empty(), exception); + fetchTimeMs, lastStableOffset, OptionalInt.empty(), error); } - public LogReadResult(Throwable e) { + public LogReadResult(Errors error) { this(new FetchDataInfo(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, MemoryRecords.EMPTY), Optional.empty(), UnifiedLog.UNKNOWN_OFFSET, @@ -75,28 +75,7 @@ public LogReadResult(Throwable e) { UnifiedLog.UNKNOWN_OFFSET, -1L, OptionalLong.empty(), - Optional.of(e)); - } - - public Errors error() { - if (exception.isPresent()) { - return Errors.forException(exception.get()); - } - return Errors.NONE; - } - - @Override - public String toString() { - return "LogReadResult(info=" + info + - ", divergingEpoch=" + divergingEpoch + - ", highWatermark=" + highWatermark + - ", leaderLogStartOffset" + leaderLogStartOffset + - ", leaderLogEndOffset" + leaderLogEndOffset + - ", followerLogStartOffset" + followerLogStartOffset + - ", fetchTimeMs=" + fetchTimeMs + - ", preferredReadReplica=" + preferredReadReplica + - ", lastStableOffset=" + lastStableOffset + - ", error=" + error() + ")"; + error); } public FetchPartitionData toFetchPartitionData(boolean isReassignmentFetch) { diff --git a/storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java b/storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java index 7620f36aebd53..d4fc0c3ef27a4 100644 --- a/storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java +++ b/storage/src/test/java/org/apache/kafka/server/purgatory/DelayedRemoteFetchTest.java @@ -469,7 +469,7 @@ private LogReadResult buildReadResult(Errors error, long highWatermark, long lea -1L, -1L, OptionalLong.empty(), - error != Errors.NONE ? Optional.of(error.exception()) : Optional.empty()); + error); } private RemoteLogReadResult buildRemoteReadResult(Errors error) { From 5184d31c9e5cedaaf22e86a57622b56f98fd38af Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 22 Oct 2025 16:07:29 +0800 Subject: [PATCH 16/17] fix conflict --- core/src/main/scala/kafka/server/DelayedRemoteFetch.scala | 0 .../org/apache/kafka/server/purgatory/DelayedRemoteFetch.java | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) delete mode 100644 core/src/main/scala/kafka/server/DelayedRemoteFetch.scala diff --git a/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala b/core/src/main/scala/kafka/server/DelayedRemoteFetch.scala deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java index 517f4390a974e..8c872811eb671 100644 --- a/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java +++ b/storage/src/main/java/org/apache/kafka/server/purgatory/DelayedRemoteFetch.java @@ -155,7 +155,7 @@ public void onComplete() { Map fetchPartitionData = new LinkedHashMap<>(); localReadResults.forEach((tpId, result) -> { CompletableFuture remoteFetchResult = remoteFetchResults.get(tpId); - if (remoteFetchInfos.containsKey(tpId) + if (remoteFetchResults.containsKey(tpId) && remoteFetchResult.isDone() && result.error() == Errors.NONE && result.info().delayedRemoteStorageFetch.isPresent()) { From ef1ce233d186e8534b5f2ece64b4de9f75dbf4f6 Mon Sep 17 00:00:00 2001 From: Lan Ding Date: Wed, 22 Oct 2025 18:30:20 +0800 Subject: [PATCH 17/17] fix comment --- .../main/scala/kafka/server/ReplicaManager.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 22589e87008b2..1d9d076a317df 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -1627,7 +1627,7 @@ class ReplicaManager(val config: KafkaConfig, private def processRemoteFetches(remoteFetchInfos: util.LinkedHashMap[TopicIdPartition, RemoteStorageFetchInfo], params: FetchParams, responseCallback: Seq[(TopicIdPartition, FetchPartitionData)] => Unit, - logReadResults: Seq[(TopicIdPartition, LogReadResult)], + logReadResults: util.LinkedHashMap[TopicIdPartition, LogReadResult], fetchPartitionStatus: Seq[(TopicIdPartition, FetchPartitionStatus)]): Unit = { val remoteFetchTasks = new util.HashMap[TopicIdPartition, Future[Void]] val remoteFetchResults = new util.HashMap[TopicIdPartition, CompletableFuture[RemoteLogReadResult]] @@ -1645,7 +1645,7 @@ class ReplicaManager(val config: KafkaConfig, remoteFetchMaxWaitMs, fetchPartitionStatus.toMap.asJava, params, - logReadResults.toMap.asJava, + logReadResults, tp => getPartitionOrException(tp), response => responseCallback(response.asScala.toSeq)) @@ -1676,7 +1676,7 @@ class ReplicaManager(val config: KafkaConfig, var hasDivergingEpoch = false var hasPreferredReadReplica = false - val logReadResultMap = new mutable.HashMap[TopicIdPartition, LogReadResult] + val logReadResultMap = new util.LinkedHashMap[TopicIdPartition, LogReadResult] logReadResults.foreach { case (topicIdPartition, logReadResult) => brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark() @@ -1712,14 +1712,15 @@ class ReplicaManager(val config: KafkaConfig, // construct the fetch results from the read results val fetchPartitionStatus = new mutable.ArrayBuffer[(TopicIdPartition, FetchPartitionStatus)] fetchInfos.foreach { case (topicIdPartition, partitionData) => - logReadResultMap.get(topicIdPartition).foreach(logReadResult => { + val logReadResult = logReadResultMap.get(topicIdPartition) + if (logReadResult != null) { val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata fetchPartitionStatus += (topicIdPartition -> new FetchPartitionStatus(logOffsetMetadata, partitionData)) - }) + } } if (!remoteFetchInfos.isEmpty) { - processRemoteFetches(remoteFetchInfos, params, responseCallback, logReadResults, fetchPartitionStatus.toSeq) + processRemoteFetches(remoteFetchInfos, params, responseCallback, logReadResultMap, fetchPartitionStatus.toSeq) } else { // If there is not enough data to respond and there is no remote data, we will let the fetch request // wait for new data.