dajac commented on code in PR #12150:
URL: https://github.com/apache/kafka/pull/12150#discussion_r874534340
##########
core/src/main/scala/kafka/cluster/Partition.scala:
##########
@@ -1133,16 +1129,94 @@ class Partition(val topicPartition: TopicPartition,
info.copy(leaderHwChange = if (leaderHWIncremented)
LeaderHwChange.Increased else LeaderHwChange.Same)
}
- def readRecords(lastFetchedEpoch: Optional[Integer],
- fetchOffset: Long,
- currentLeaderEpoch: Optional[Integer],
- maxBytes: Int,
- fetchIsolation: FetchIsolation,
- fetchOnlyFromLeader: Boolean,
- minOneMessage: Boolean): LogReadInfo =
inReadLock(leaderIsrUpdateLock) {
- // decide whether to only fetch from leader
- val localLog = localLogWithEpochOrException(currentLeaderEpoch,
fetchOnlyFromLeader)
+ def fetchRecords(
+ fetchParams: FetchParams,
+ fetchPartitionData: FetchRequest.PartitionData,
+ fetchTimeMs: Long,
+ maxBytes: Int,
+ minOneMessage: Boolean,
+ updateFetchState: Boolean
+ ): LogReadInfo = {
+ def doReadRecords(log: UnifiedLog): LogReadInfo = {
+ readRecords(
+ log,
+ fetchPartitionData.lastFetchedEpoch,
+ fetchPartitionData.fetchOffset,
+ fetchPartitionData.currentLeaderEpoch,
+ maxBytes,
+ fetchParams.isolation,
+ minOneMessage
+ )
+ }
+
+ if (fetchParams.isFromFollower) {
+ var replica: Replica = null
+
+ val logReadInfo = inReadLock(leaderIsrUpdateLock) {
+ val localLog =
localLogWithEpochOrException(fetchPartitionData.currentLeaderEpoch,
fetchParams.fetchOnlyLeader)
+ replica = followerReplicaOrThrow(fetchParams.replicaId,
fetchPartitionData)
Review Comment:
Do we really need to get the replica while holding the `leaderIsrUpdateLock`
lock? I suppose that it does not hurt us.
##########
core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala:
##########
@@ -366,4 +332,33 @@ class ReplicaManagerQuotasTest {
when(quota.isThrottled(any[TopicPartition])).thenReturn(true)
quota
}
+
+ private def buildFollowerFetchParams(
Review Comment:
nit: We have the same helper methods in `PartitionTest`. I wonder if we
could reuse them. We could perhaps move them to the companion object of
`PartitionTest` and reuse them here as well. What do you think?
##########
core/src/main/scala/kafka/cluster/Partition.scala:
##########
@@ -1133,16 +1129,94 @@ class Partition(val topicPartition: TopicPartition,
info.copy(leaderHwChange = if (leaderHWIncremented)
LeaderHwChange.Increased else LeaderHwChange.Same)
}
- def readRecords(lastFetchedEpoch: Optional[Integer],
- fetchOffset: Long,
- currentLeaderEpoch: Optional[Integer],
- maxBytes: Int,
- fetchIsolation: FetchIsolation,
- fetchOnlyFromLeader: Boolean,
- minOneMessage: Boolean): LogReadInfo =
inReadLock(leaderIsrUpdateLock) {
- // decide whether to only fetch from leader
- val localLog = localLogWithEpochOrException(currentLeaderEpoch,
fetchOnlyFromLeader)
+ def fetchRecords(
+ fetchParams: FetchParams,
+ fetchPartitionData: FetchRequest.PartitionData,
+ fetchTimeMs: Long,
+ maxBytes: Int,
+ minOneMessage: Boolean,
+ updateFetchState: Boolean
+ ): LogReadInfo = {
+ def doReadRecords(log: UnifiedLog): LogReadInfo = {
+ readRecords(
+ log,
+ fetchPartitionData.lastFetchedEpoch,
+ fetchPartitionData.fetchOffset,
+ fetchPartitionData.currentLeaderEpoch,
+ maxBytes,
+ fetchParams.isolation,
+ minOneMessage
+ )
+ }
+
+ if (fetchParams.isFromFollower) {
+ var replica: Replica = null
+
+ val logReadInfo = inReadLock(leaderIsrUpdateLock) {
+ val localLog =
localLogWithEpochOrException(fetchPartitionData.currentLeaderEpoch,
fetchParams.fetchOnlyLeader)
Review Comment:
nit: Should we rename `localLogWithEpochOrException` to
`localLogWithEpochOrThrow` to be consistent with `followerReplicaOrThrow`?
##########
core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala:
##########
@@ -366,4 +332,33 @@ class ReplicaManagerQuotasTest {
when(quota.isThrottled(any[TopicPartition])).thenReturn(true)
quota
}
+
+ private def buildFollowerFetchParams(
+ replicaId: Int,
+ ): FetchParams = {
+ FetchParams(
+ requestVersion = ApiKeys.FETCH.latestVersion,
+ replicaId = replicaId,
+ maxWaitMs = 0,
+ minBytes = 1,
+ maxBytes = Int.MaxValue,
+ isolation = FetchLogEnd,
+ clientMetadata = None
+ )
+ }
+
+ private def buildConsumerFetchParams(
+
Review Comment:
nit: This empty line is not needed.
##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -439,18 +545,33 @@ class PartitionTest extends AbstractPartitionTest {
val leaderEpoch = 5
val partition = setupPartitionWithMocks(leaderEpoch, isLeader = false)
- def assertReadRecordsError(error: Errors,
- currentLeaderEpochOpt: Optional[Integer],
- fetchOnlyLeader: Boolean): Unit = {
+ def assertReadRecordsError(
Review Comment:
nit: Is it worth consolidating this one with `assertReadRecordsError` in the
previous test? There are identical except that this one takes `clientMetadata`
as additional parameter.
##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -412,14 +506,26 @@ class PartitionTest extends AbstractPartitionTest {
def assertReadRecordsError(error: Errors,
currentLeaderEpochOpt: Optional[Integer]): Unit
= {
try {
- partition.readRecords(
- lastFetchedEpoch = Optional.empty(),
- fetchOffset = 0L,
- currentLeaderEpoch = currentLeaderEpochOpt,
+ val fetchOffset = 0L
+ val logStartOffset = 0L
+
+ val fetchPartitionData = new FetchRequest.PartitionData(
Review Comment:
nit: Could we reuse `fetchFollower` here as well?
##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -83,14 +91,22 @@ class PartitionTest extends AbstractPartitionTest {
}
def read(lastFetchedEpoch: Int, fetchOffset: Long): LogReadInfo = {
- partition.readRecords(
- Optional.of(lastFetchedEpoch),
+ val fetchPartitionData = new FetchRequest.PartitionData(
Review Comment:
nit: Could we reuse `fetchFollower` here?
##########
core/src/test/scala/unit/kafka/cluster/PartitionTest.scala:
##########
@@ -412,14 +506,26 @@ class PartitionTest extends AbstractPartitionTest {
def assertReadRecordsError(error: Errors,
currentLeaderEpochOpt: Optional[Integer]): Unit
= {
try {
- partition.readRecords(
- lastFetchedEpoch = Optional.empty(),
- fetchOffset = 0L,
- currentLeaderEpoch = currentLeaderEpochOpt,
+ val fetchOffset = 0L
+ val logStartOffset = 0L
+
+ val fetchPartitionData = new FetchRequest.PartitionData(
+ Uuid.ZERO_UUID,
+ fetchOffset,
+ logStartOffset,
+ Int.MaxValue,
+ currentLeaderEpochOpt,
+ Optional.empty[Integer]
+ )
+
+ partition.fetchRecords(
+ fetchParams = followerFetchParams(remoteReplicaId),
+ fetchPartitionData = fetchPartitionData,
+ fetchTimeMs = time.milliseconds(),
maxBytes = 1024,
- fetchIsolation = FetchLogEnd,
- fetchOnlyFromLeader = true,
- minOneMessage = false)
+ minOneMessage = false,
+ updateFetchState = true
+ )
if (error != Errors.NONE)
fail(s"Expected readRecords to fail with error $error")
} catch {
Review Comment:
Not related to your change but it seems that we should have used
`assertThrows` here.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]