-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-9039: Optimize ReplicaFetcher fetch path #7443
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d2a13d8
8545888
29fdd60
0e57e3e
bbb1e99
2614b24
f0bf23a
32e30ac
40728b3
1a0ed01
4a306ad
1aa07f4
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ import java.util.Optional | |
| import kafka.api._ | ||
| import kafka.cluster.BrokerEndPoint | ||
| import kafka.log.LogAppendInfo | ||
| import kafka.server.AbstractFetcherThread.ReplicaFetch | ||
| import kafka.server.AbstractFetcherThread.ResultWithPartitions | ||
| import org.apache.kafka.clients.FetchSessionHandler | ||
| import org.apache.kafka.common.TopicPartition | ||
|
|
@@ -34,7 +35,7 @@ import org.apache.kafka.common.requests._ | |
| import org.apache.kafka.common.utils.{LogContext, Time} | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.{Map, mutable} | ||
| import scala.collection.{mutable, Map} | ||
|
|
||
| class ReplicaFetcherThread(name: String, | ||
| fetcherId: Int, | ||
|
|
@@ -96,12 +97,16 @@ class ReplicaFetcherThread(name: String, | |
| private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes | ||
| private val fetchSize = brokerConfig.replicaFetchMaxBytes | ||
| private val brokerSupportsLeaderEpochRequest = brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV2 | ||
| private val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) | ||
| val fetchSessionHandler = new FetchSessionHandler(logContext, sourceBroker.id) | ||
|
|
||
| override protected def latestEpoch(topicPartition: TopicPartition): Option[Int] = { | ||
| replicaMgr.localLogOrException(topicPartition).latestEpoch | ||
| } | ||
|
|
||
| override protected def logStartOffset(topicPartition: TopicPartition): Long = { | ||
| replicaMgr.localLogOrException(topicPartition).logStartOffset | ||
| } | ||
|
|
||
| override protected def logEndOffset(topicPartition: TopicPartition): Long = { | ||
| replicaMgr.localLogOrException(topicPartition).logEndOffset | ||
| } | ||
|
|
@@ -191,14 +196,14 @@ class ReplicaFetcherThread(name: String, | |
| } | ||
|
|
||
|
|
||
| override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, FetchData)] = { | ||
| override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Map[TopicPartition, FetchData] = { | ||
| try { | ||
| val clientResponse = leaderEndpoint.sendRequest(fetchRequest) | ||
| val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse[Records]] | ||
| if (!fetchSessionHandler.handleResponse(fetchResponse)) { | ||
| Nil | ||
| Map.empty | ||
| } else { | ||
| fetchResponse.responseData.asScala.toSeq | ||
| fetchResponse.responseData.asScala | ||
| } | ||
| } catch { | ||
| case t: Throwable => | ||
|
|
@@ -236,15 +241,15 @@ class ReplicaFetcherThread(name: String, | |
| } | ||
| } | ||
|
|
||
| override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[FetchRequest.Builder]] = { | ||
| override def buildFetch(partitionMap: Map[TopicPartition, PartitionFetchState]): ResultWithPartitions[Option[ReplicaFetch]] = { | ||
| val partitionsWithError = mutable.Set[TopicPartition]() | ||
|
|
||
| val builder = fetchSessionHandler.newBuilder() | ||
| val builder = fetchSessionHandler.newBuilder(partitionMap.size, false) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Use the new builder which presizes the PartitionData hashmap, and which does not make another copy of sessionPartitions. We will not be modifying sessionPartitions, or using it again after we have processed the fetch request, so we do not need to make an unnecessary copy.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm, here, we choose not to copy sessionPartitions in the builder. However, in line 265 and line 273, we still read sessionPartitions. Is that correct?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I believe this is still correct. We do not copy sessionPartitions, but sessionPartitions will also not be modified until the next fetch/doWork iteration, and all use of it is from the same thread. I believe it is thus safe to use it without making a copy.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for the explanation. That makes sense. |
||
| partitionMap.foreach { case (topicPartition, fetchState) => | ||
| // We will not include a replica in the fetch request if it should be throttled. | ||
| if (fetchState.isReadyForFetch && !shouldFollowerThrottle(quota, topicPartition)) { | ||
| if (fetchState.isReadyForFetch && !shouldFollowerThrottle(quota, fetchState, topicPartition)) { | ||
| try { | ||
| val logStartOffset = replicaMgr.localLogOrException(topicPartition).logStartOffset | ||
| val logStartOffset = this.logStartOffset(topicPartition) | ||
| builder.add(topicPartition, new FetchRequest.PartitionData( | ||
| fetchState.fetchOffset, logStartOffset, fetchSize, Optional.of(fetchState.currentLeaderEpoch))) | ||
| } catch { | ||
|
|
@@ -265,7 +270,7 @@ class ReplicaFetcherThread(name: String, | |
| .setMaxBytes(maxBytes) | ||
| .toForget(fetchData.toForget) | ||
| .metadata(fetchData.metadata) | ||
| Some(requestBuilder) | ||
| Some(ReplicaFetch(fetchData.sessionPartitions(), requestBuilder)) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. By passing through sessionPartitions we can avoid making another copy of the PartitionMap in https://github.com/apache/kafka/pull/7443/files#diff-2d03a5d4349a23e56c09b4e773055249L119. sessionPartitions contains all of the PartitionData that we need to process the fetch response. |
||
| } | ||
|
|
||
| ResultWithPartitions(fetchRequestOpt, partitionsWithError) | ||
|
|
@@ -330,9 +335,8 @@ class ReplicaFetcherThread(name: String, | |
| * To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list, | ||
| * the quota is exceeded and the replica is not in sync. | ||
| */ | ||
| private def shouldFollowerThrottle(quota: ReplicaQuota, topicPartition: TopicPartition): Boolean = { | ||
| val isReplicaInSync = fetcherLagStats.isReplicaInSync(topicPartition) | ||
| !isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded | ||
| private def shouldFollowerThrottle(quota: ReplicaQuota, fetchState: PartitionFetchState, topicPartition: TopicPartition): Boolean = { | ||
| !fetchState.isReplicaInSync && quota.isThrottled(topicPartition) && quota.isQuotaExceeded | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We avoid a HashMap lookup here by including the lag in the PartitionFetchState directly. |
||
| } | ||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'd suggest we make one step further, to just make the
sessionPartitionsa local variable only: this is because in later handling logic after the response is sent back, we only needsessionPartitions.keySet / sizeonly, which we could maintain separately as a member field which is populated after the build() call. WDYT?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
NVM I realized now you also need the
sessionPartitionsfor saving one fetch state copying as well.