jeffkbkim commented on code in PR #12783: URL: https://github.com/apache/kafka/pull/12783#discussion_r1007253784
########## core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala: ########## @@ -0,0 +1,97 @@ +/** + * 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, BrokerTopicStats, 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 +import org.junit.jupiter.api.{AfterEach, BeforeEach, Timeout} +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.ValueSource + +import java.util.Properties + +class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest { + val numNodes = 2 + val numParts = 1 + val initialMessages = 100 + val nMessages = 100 + + 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)) + + @BeforeEach + def initializeFetchFromFollowerCluster(): Unit = { + // Create a 2 broker cluster where broker 0 is the leader and 1 is the follower. + + admin = TestUtils.createAdminClient(brokers, listenerName) + TestUtils.createTopicWithAdminRaw( + admin, + topic, + replicaAssignment = Map(0 -> Seq(leaderBrokerId, followerBrokerId)) + ) + TestUtils.generateAndProduceMessages(brokers, topic, initialMessages) + } + + @AfterEach + def close(): Unit = { + if (admin != null) { + admin.close() + } + } + + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumName) + @ValueSource(strings = Array("zk", "kraft")) + @Timeout(30) + def testFollowerCompleteDelayedPurgatoryOnReplication(quorum: String): Unit = { + TestUtils.generateAndProduceMessages(brokers, topic, nMessages) + // set fetch.max.wait.ms to a value (45 seconds) greater than the timeout (30 seconds) to ensure that the + // test only passes when the delayed fetch purgatory is completed after successfully replicating from the leader. + + val totalMessages = initialMessages + nMessages + val topicPartition = new TopicPartition(topic, 0) + val offsetMap = Map[TopicPartition, Long]( + topicPartition -> (totalMessages - 1) Review Comment: the suggestion makes sense. i think the point you're getting at is we need to have the fetch sit in purgatory before replication. and i confirmed your suggestion works locally. however, i'm still unable to grasp why the current logic fails. let's say "there is no record at offset 200 yet" which means replication is not complete. the fetch request will sit in the purgatory and wake up once the record is replicated. could it be the case where record at offset 200 is already replicated but the follower's HWM is not updated yet to 200 so the fetch request sits in purgatory and never wakes up because there are no new records? -- 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