dajac commented on code in PR #12783:
URL: https://github.com/apache/kafka/pull/12783#discussion_r1012620903


##########
core/src/main/scala/kafka/log/UnifiedLog.scala:
##########
@@ -433,6 +433,22 @@ class UnifiedLog(@volatile var logStartOffset: Long,
     }
   }
 
+  /**

Review Comment:
   nit: Could we also update the scaladoc of `def updateHighWatermark(hw: 
Long)` to remove the part about the follower updating the HWM. This is no 
longer the case.



##########
core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala:
##########
@@ -0,0 +1,95 @@
+/**
+ * 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 integration.kafka.server
+
+import kafka.server.{BaseFetchRequestTest, KafkaConfig}
+import kafka.utils.{TestInfoUtils, TestUtils}
+import org.apache.kafka.clients.admin.Admin
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.FetchResponse
+import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.{BeforeEach, Timeout}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.ValueSource
+
+import java.util.Properties
+import scala.jdk.CollectionConverters._
+
+class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
+  val numNodes = 2
+  val numParts = 1
+
+  val topic = "test-fetch-from-follower"
+  val leaderBrokerId = 0
+  val followerBrokerId = 1
+  var admin: Admin = null
+
+  def overridingProps: Properties = {
+    val props = new Properties
+    props.put(KafkaConfig.NumPartitionsProp, numParts.toString)
+    props
+  }
+
+  override def generateConfigs: collection.Seq[KafkaConfig] = {
+    TestUtils.createBrokerConfigs(numNodes, zkConnectOrNull, 
enableControlledShutdown = false, enableFetchFromFollower = true)
+      .map(KafkaConfig.fromProps(_, overridingProps))
+  }
+
+  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+  @ValueSource(strings = Array("zk", "kraft"))
+  @Timeout(15)
+  def testFollowerCompleteDelayedFetchesOnReplication(quorum: String): Unit = {
+    // Create a 2 broker cluster where broker 0 is the leader and 1 is the 
follower.
+    admin = createAdminClient()
+    TestUtils.createTopicWithAdmin(
+      admin,
+      topic,
+      brokers,
+      replicaAssignment = Map(0 -> Seq(leaderBrokerId, followerBrokerId))
+    )
+
+    // Set fetch.max.wait.ms to a value (20 seconds) greater than the timeout 
(15 seconds).
+    // Send a fetch request before the record is replicated to ensure that the 
replication
+    // triggers purgatory completion.

Review Comment:
   nit: Should we move this to right before `val fetchRequest` as the comment 
is for this part?



##########
core/src/main/scala/kafka/server/ReplicaFetcherThread.scala:
##########
@@ -132,9 +139,22 @@ class ReplicaFetcherThread(name: String,
 
     brokerTopicStats.updateReplicationBytesIn(records.sizeInBytes)
 
+    val highWatermarkChanged = 
log.maybeUpdateHighWatermark(partitionData.highWatermark)
+    if (highWatermarkChanged) {
+      logAppendInfo.foreach { _ => partitionsWithNewHighWatermark += 
topicPartition }
+      if (logTrace)
+        trace(s"Follower updated replica high watermark for partition 
$topicPartition to ${partitionData.highWatermark}")

Review Comment:
   I see. How about returning an `Option` in `maybeUpdateHighWatermark` which 
would contain the updated HWM only if it was changed. Would this work?
   
   I would also update the trace message to include both the received HWM from 
the leader and the updated HWM. Both seems useful here.



##########
core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala:
##########
@@ -0,0 +1,95 @@
+/**
+ * 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 integration.kafka.server
+
+import kafka.server.{BaseFetchRequestTest, KafkaConfig}
+import kafka.utils.{TestInfoUtils, TestUtils}
+import org.apache.kafka.clients.admin.Admin
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.requests.FetchResponse
+import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.{BeforeEach, Timeout}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.ValueSource
+
+import java.util.Properties
+import scala.jdk.CollectionConverters._
+
+class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
+  val numNodes = 2
+  val numParts = 1
+
+  val topic = "test-fetch-from-follower"
+  val leaderBrokerId = 0
+  val followerBrokerId = 1
+  var admin: Admin = null
+
+  def overridingProps: Properties = {
+    val props = new Properties
+    props.put(KafkaConfig.NumPartitionsProp, numParts.toString)
+    props
+  }
+
+  override def generateConfigs: collection.Seq[KafkaConfig] = {
+    TestUtils.createBrokerConfigs(numNodes, zkConnectOrNull, 
enableControlledShutdown = false, enableFetchFromFollower = true)
+      .map(KafkaConfig.fromProps(_, overridingProps))
+  }
+
+  @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName)
+  @ValueSource(strings = Array("zk", "kraft"))
+  @Timeout(15)
+  def testFollowerCompleteDelayedFetchesOnReplication(quorum: String): Unit = {
+    // Create a 2 broker cluster where broker 0 is the leader and 1 is the 
follower.

Review Comment:
   nit: Should we say `Create a topic with 2 replicas where...`? The cluster is 
automatically created.



##########
core/src/test/scala/unit/kafka/server/FetchRequestTest.scala:
##########
@@ -51,7 +51,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
 
     def createFetchRequest(topicPartitions: Seq[TopicPartition], offsetMap: 
Map[TopicPartition, Long] = Map.empty,

Review Comment:
   Thoughts?



##########
core/src/main/scala/kafka/server/ReplicaFetcherThread.scala:
##########
@@ -132,9 +139,22 @@ class ReplicaFetcherThread(name: String,
 
     brokerTopicStats.updateReplicationBytesIn(records.sizeInBytes)
 
+    val highWatermarkChanged = 
log.maybeUpdateHighWatermark(partitionData.highWatermark)
+    if (highWatermarkChanged) {
+      logAppendInfo.foreach { _ => partitionsWithNewHighWatermark += 
topicPartition }

Review Comment:
   nit: I think that we don't have to do `logAppendInfo.foreach` here anymore, 
right? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to