@@ -56,16 +56,16 @@ import org.apache.kafka.server.config.ReplicationConfigs
5656import org .apache .kafka .server .log .remote .storage .RemoteLogManager
5757import org .apache .kafka .server .metrics .KafkaMetricsGroup
5858import org .apache .kafka .server .network .BrokerEndPoint
59- import org .apache .kafka .server .purgatory .{DelayedDeleteRecords , DelayedOperationPurgatory , DelayedRemoteListOffsets , DeleteRecordsPartitionStatus , ListOffsetsPartitionStatus , TopicPartitionOperationKey }
59+ import org .apache .kafka .server .purgatory .{DelayedDeleteRecords , DelayedOperationPurgatory , DelayedRemoteFetch , DelayedRemoteListOffsets , DeleteRecordsPartitionStatus , ListOffsetsPartitionStatus , TopicPartitionOperationKey }
6060import org .apache .kafka .server .share .fetch .{DelayedShareFetchKey , DelayedShareFetchPartitionKey }
6161import org .apache .kafka .server .storage .log .{FetchParams , FetchPartitionData }
6262import org .apache .kafka .server .transaction .AddPartitionsToTxnManager
6363import org .apache .kafka .server .transaction .AddPartitionsToTxnManager .TransactionSupportedOperation
6464import org .apache .kafka .server .util .timer .{SystemTimer , TimerTask }
6565import org .apache .kafka .server .util .{Scheduler , ShutdownableThread }
66- import org .apache .kafka .server .{ActionQueue , DelayedActionQueue , LogReadResult , common }
66+ import org .apache .kafka .server .{ActionQueue , DelayedActionQueue , common }
6767import org .apache .kafka .storage .internals .checkpoint .{LazyOffsetCheckpoints , OffsetCheckpointFile , OffsetCheckpoints }
68- import org .apache .kafka .storage .internals .log .{AppendOrigin , FetchDataInfo , LeaderHwChange , LogAppendInfo , LogConfig , LogDirFailureChannel , LogOffsetMetadata , LogReadInfo , OffsetResultHolder , RecordValidationException , RemoteLogReadResult , RemoteStorageFetchInfo , UnifiedLog , VerificationGuard }
68+ import org .apache .kafka .storage .internals .log .{AppendOrigin , FetchDataInfo , FetchPartitionStatus , LeaderHwChange , LogAppendInfo , LogConfig , LogDirFailureChannel , LogOffsetMetadata , LogReadInfo , LogReadResult , OffsetResultHolder , RecordValidationException , RemoteLogReadResult , RemoteStorageFetchInfo , UnifiedLog , VerificationGuard }
6969import org .apache .kafka .storage .log .metrics .BrokerTopicStats
7070
7171import java .io .File
@@ -188,19 +188,7 @@ object ReplicaManager {
188188 - 1L ,
189189 - 1L ,
190190 OptionalLong .empty(),
191- Optional .of(e))
192- }
193-
194- def createLogReadResult (e : Throwable ): LogReadResult = {
195- new LogReadResult (new FetchDataInfo (LogOffsetMetadata .UNKNOWN_OFFSET_METADATA , MemoryRecords .EMPTY ),
196- Optional .empty(),
197- UnifiedLog .UNKNOWN_OFFSET ,
198- UnifiedLog .UNKNOWN_OFFSET ,
199- UnifiedLog .UNKNOWN_OFFSET ,
200- UnifiedLog .UNKNOWN_OFFSET ,
201- - 1L ,
202- OptionalLong .empty(),
203- Optional .of(e))
191+ Errors .forException(e));
204192 }
205193
206194 private [server] def isListOffsetsTimestampUnsupported (timestamp : JLong , version : Short ): Boolean = {
@@ -1639,7 +1627,7 @@ class ReplicaManager(val config: KafkaConfig,
16391627 private def processRemoteFetches (remoteFetchInfos : util.LinkedHashMap [TopicIdPartition , RemoteStorageFetchInfo ],
16401628 params : FetchParams ,
16411629 responseCallback : Seq [(TopicIdPartition , FetchPartitionData )] => Unit ,
1642- logReadResults : Seq [( TopicIdPartition , LogReadResult ) ],
1630+ logReadResults : util. LinkedHashMap [ TopicIdPartition , LogReadResult ],
16431631 fetchPartitionStatus : Seq [(TopicIdPartition , FetchPartitionStatus )]): Unit = {
16441632 val remoteFetchTasks = new util.HashMap [TopicIdPartition , Future [Void ]]
16451633 val remoteFetchResults = new util.HashMap [TopicIdPartition , CompletableFuture [RemoteLogReadResult ]]
@@ -1651,8 +1639,15 @@ class ReplicaManager(val config: KafkaConfig,
16511639 }
16521640
16531641 val remoteFetchMaxWaitMs = config.remoteLogManagerConfig.remoteFetchMaxWaitMs().toLong
1654- val remoteFetch = new DelayedRemoteFetch (remoteFetchTasks, remoteFetchResults, remoteFetchInfos, remoteFetchMaxWaitMs,
1655- fetchPartitionStatus, params, logReadResults, this , responseCallback)
1642+ val remoteFetch = new DelayedRemoteFetch (remoteFetchTasks,
1643+ remoteFetchResults,
1644+ remoteFetchInfos,
1645+ remoteFetchMaxWaitMs,
1646+ fetchPartitionStatus.toMap.asJava,
1647+ params,
1648+ logReadResults,
1649+ tp => getPartitionOrException(tp),
1650+ response => responseCallback(response.asScala.toSeq))
16561651
16571652 // create a list of (topic, partition) pairs to use as keys for this delayed fetch operation
16581653 val delayedFetchKeys = remoteFetchTasks.asScala.map { case (tp, _) => new TopicPartitionOperationKey (tp) }.toList
@@ -1681,7 +1676,7 @@ class ReplicaManager(val config: KafkaConfig,
16811676
16821677 var hasDivergingEpoch = false
16831678 var hasPreferredReadReplica = false
1684- val logReadResultMap = new mutable. HashMap [TopicIdPartition , LogReadResult ]
1679+ val logReadResultMap = new util. LinkedHashMap [TopicIdPartition , LogReadResult ]
16851680
16861681 logReadResults.foreach { case (topicIdPartition, logReadResult) =>
16871682 brokerTopicStats.topicStats(topicIdPartition.topicPartition.topic).totalFetchRequestRate.mark()
@@ -1717,14 +1712,15 @@ class ReplicaManager(val config: KafkaConfig,
17171712 // construct the fetch results from the read results
17181713 val fetchPartitionStatus = new mutable.ArrayBuffer [(TopicIdPartition , FetchPartitionStatus )]
17191714 fetchInfos.foreach { case (topicIdPartition, partitionData) =>
1720- logReadResultMap.get(topicIdPartition).foreach(logReadResult => {
1715+ val logReadResult = logReadResultMap.get(topicIdPartition)
1716+ if (logReadResult != null ) {
17211717 val logOffsetMetadata = logReadResult.info.fetchOffsetMetadata
1722- fetchPartitionStatus += (topicIdPartition -> FetchPartitionStatus (logOffsetMetadata, partitionData))
1723- })
1718+ fetchPartitionStatus += (topicIdPartition -> new FetchPartitionStatus (logOffsetMetadata, partitionData))
1719+ }
17241720 }
17251721
17261722 if (! remoteFetchInfos.isEmpty) {
1727- processRemoteFetches(remoteFetchInfos, params, responseCallback, logReadResults , fetchPartitionStatus.toSeq)
1723+ processRemoteFetches(remoteFetchInfos, params, responseCallback, logReadResultMap , fetchPartitionStatus.toSeq)
17281724 } else {
17291725 // If there is not enough data to respond and there is no remote data, we will let the fetch request
17301726 // wait for new data.
@@ -1812,7 +1808,7 @@ class ReplicaManager(val config: KafkaConfig,
18121808 - 1L ,
18131809 OptionalLong .of(offsetSnapshot.lastStableOffset.messageOffset),
18141810 if (preferredReadReplica.isDefined) OptionalInt .of(preferredReadReplica.get) else OptionalInt .empty(),
1815- Optional .empty() )
1811+ Errors . NONE )
18161812 } else {
18171813 log = partition.localLogWithEpochOrThrow(fetchInfo.currentLeaderEpoch, params.fetchOnlyLeader())
18181814
@@ -1836,7 +1832,7 @@ class ReplicaManager(val config: KafkaConfig,
18361832 fetchTimeMs,
18371833 OptionalLong .of(readInfo.lastStableOffset),
18381834 if (preferredReadReplica.isDefined) OptionalInt .of(preferredReadReplica.get) else OptionalInt .empty(),
1839- Optional .empty()
1835+ Errors . NONE
18401836 )
18411837 }
18421838 } catch {
@@ -1849,7 +1845,7 @@ class ReplicaManager(val config: KafkaConfig,
18491845 _ : ReplicaNotAvailableException |
18501846 _ : KafkaStorageException |
18511847 _ : InconsistentTopicIdException ) =>
1852- createLogReadResult(e )
1848+ new LogReadResult ( Errors .forException(e) )
18531849 case e : OffsetOutOfRangeException =>
18541850 handleOffsetOutOfRangeError(tp, params, fetchInfo, adjustedMaxBytes, minOneMessage, log, fetchTimeMs, e)
18551851 case e : Throwable =>
@@ -1868,7 +1864,7 @@ class ReplicaManager(val config: KafkaConfig,
18681864 UnifiedLog .UNKNOWN_OFFSET ,
18691865 - 1L ,
18701866 OptionalLong .empty(),
1871- Optional .of (e)
1867+ Errors .forException (e)
18721868 )
18731869 }
18741870 }
@@ -1949,10 +1945,10 @@ class ReplicaManager(val config: KafkaConfig,
19491945 fetchInfo.logStartOffset,
19501946 fetchTimeMs,
19511947 OptionalLong .of(log.lastStableOffset),
1952- Optional .empty[ Throwable ]() )
1948+ Errors . NONE )
19531949 }
19541950 } else {
1955- createLogReadResult( exception)
1951+ new LogReadResult ( Errors .forException( exception) )
19561952 }
19571953 }
19581954
0 commit comments