dengziming commented on a change in pull request #9769: URL: https://github.com/apache/kafka/pull/9769#discussion_r680328486
########## File path: core/src/test/scala/integration/kafka/server/MetadataRequestBetweenDifferentIbpTest.scala ########## @@ -0,0 +1,102 @@ +/** + * 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 kafka.server + +import kafka.api.{ApiVersion, KAFKA_2_8_IV0, KAFKA_3_0_IV1} +import kafka.network.SocketServer +import kafka.utils.TestUtils +import kafka.zk.ZkVersion +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.message.MetadataRequestData +import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} +import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Test + +import scala.collection.{Map, Seq} + +class MetadataRequestBetweenDifferentIbpTest extends BaseRequestTest { + + override def brokerCount: Int = 3 + override def generateConfigs: Seq[KafkaConfig] = { + Seq( + createConfig(0, KAFKA_2_8_IV0), + createConfig(1, KAFKA_3_0_IV1), Review comment: In fact, when I change this to KAFKA_3_1_IV0, this test became a flaky test, I think this may be related to the code about fetch by topicId you added previously, I am still investigating. below is the exception throws: [2021-07-31 16:15:29,658] WARN maxCnxns is not configured, using default value 0. (org.apache.zookeeper.server.ServerCnxnFactory:309) [2021-07-31 16:15:30,540] WARN No meta.properties file under dir /var/folders/xs/1lh3bwpj2674ch_3wyqbcv6c0000gn/T/kafka-3222035436551046976/meta.properties (kafka.server.BrokerMetadataCheckpoint:70) [2021-07-31 16:15:32,153] WARN Error while loading kafka-version.properties: null (org.apache.kafka.common.utils.AppInfoParser:46) [2021-07-31 16:15:32,270] WARN No meta.properties file under dir /var/folders/xs/1lh3bwpj2674ch_3wyqbcv6c0000gn/T/kafka-4473390687426195767/meta.properties (kafka.server.BrokerMetadataCheckpoint:70) [2021-07-31 16:15:32,620] WARN No meta.properties file under dir /var/folders/xs/1lh3bwpj2674ch_3wyqbcv6c0000gn/T/kafka-3222063565575329695/meta.properties (kafka.server.BrokerMetadataCheckpoint:70) [2021-07-31 16:15:34,739] WARN [Controller id=0] A controller has been elected but just resigned, this will result in another round of election (kafka.controller.KafkaController:72) org.apache.kafka.common.errors.ControllerMovedException: The ephemeral node at /controller went away while checking whether the controller election succeeds. Aborting controller startup procedure [2021-07-31 16:15:34,822] WARN [Controller id=2] A controller has been elected but just resigned, this will result in another round of election (kafka.controller.KafkaController:72) org.apache.kafka.common.errors.ControllerMovedException: Controller epoch zkVersion check fails. Expected zkVersion = 4 [2021-07-31 16:15:35,087] WARN [ReplicaFetcher replicaId=2, leaderId=1, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=2, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=289640652, epoch=1), rackId=) (kafka.server.ReplicaFetcherThread:72) org.apache.kafka.common.errors.FetchSessionTopicIdException: The fetch session encountered inconsistent topic ID usage [2021-07-31 16:15:35,087] WARN [ReplicaFetcher replicaId=1, leaderId=2, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=1, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=125490650, epoch=1), rackId=) (kafka.server.ReplicaFetcherThread:72) org.apache.kafka.common.errors.FetchSessionTopicIdException: The fetch session encountered inconsistent topic ID usage [2021-07-31 16:15:50,273] WARN [KafkaServer id=2] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,273] WARN [KafkaServer id=0] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,277] WARN [KafkaServer id=1] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,383] WARN [KafkaServer id=0] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,384] WARN [KafkaServer id=2] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,386] WARN [KafkaServer id=1] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,493] WARN [KafkaServer id=2] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,495] WARN [KafkaServer id=1] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,495] WARN [KafkaServer id=0] Retrying controlled shutdown after the previous attempt failed... (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,526] WARN [KafkaServer id=2] Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,527] WARN [KafkaServer id=0] Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,526] WARN [KafkaServer id=1] Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed (kafka.server.KafkaServer:70) [2021-07-31 16:15:50,586] WARN [ReplicaFetcher replicaId=2, leaderId=1, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=2, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=297980525, epoch=27), rackId=) (kafka.server.ReplicaFetcherThread:72) java.io.IOException: Connection to 1 was disconnected before the response was read at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:99) at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:109) at kafka.server.ReplicaFetcherThread.fetchFromLeader(ReplicaFetcherThread.scala:219) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:314) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:137) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:136) at scala.Option.foreach(Option.scala:437) at kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:136) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:119) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2021-07-31 16:15:50,587] WARN [ReplicaFetcher replicaId=0, leaderId=2, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=0, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=1654389138, epoch=30), rackId=) (kafka.server.ReplicaFetcherThread:72) java.io.IOException: Connection to 2 was disconnected before the response was read at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:99) at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:109) at kafka.server.ReplicaFetcherThread.fetchFromLeader(ReplicaFetcherThread.scala:219) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:314) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:137) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:136) at scala.Option.foreach(Option.scala:437) at kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:136) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:119) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2021-07-31 16:15:50,587] WARN [ReplicaFetcher replicaId=1, leaderId=2, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=1, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=1562647001, epoch=27), rackId=) (kafka.server.ReplicaFetcherThread:72) java.io.IOException: Connection to 2 was disconnected before the response was read at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:99) at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:109) at kafka.server.ReplicaFetcherThread.fetchFromLeader(ReplicaFetcherThread.scala:219) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:314) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:137) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:136) at scala.Option.foreach(Option.scala:437) at kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:136) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:119) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2021-07-31 16:15:50,587] WARN [ReplicaFetcher replicaId=0, leaderId=1, fetcherId=0] Error in response for fetch request (type=FetchRequest, replicaId=0, maxWait=500, minBytes=1, maxBytes=10485760, fetchData={}, isolationLevel=READ_UNCOMMITTED, toForget=, metadata=(sessionId=1980753179, epoch=30), rackId=) (kafka.server.ReplicaFetcherThread:72) java.io.IOException: Connection to 1 was disconnected before the response was read at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:99) at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:109) at kafka.server.ReplicaFetcherThread.fetchFromLeader(ReplicaFetcherThread.scala:219) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:314) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3(AbstractFetcherThread.scala:137) at kafka.server.AbstractFetcherThread.$anonfun$maybeFetch$3$adapted(AbstractFetcherThread.scala:136) at scala.Option.foreach(Option.scala:437) at kafka.server.AbstractFetcherThread.maybeFetch(AbstractFetcherThread.scala:136) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:119) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:96) [2021-07-31 16:15:51,480] WARN Unexpected exception (org.apache.zookeeper.server.NIOServerCnxn:364) EndOfStreamException: Unable to read additional data from client, it probably closed the socket: address = /127.0.0.1:62546, session = 0x1002345a02e0001 at org.apache.zookeeper.server.NIOServerCnxn.handleFailedRead(NIOServerCnxn.java:163) at org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:326) at org.apache.zookeeper.server.NIOServerCnxnFactory$IOWorkRequest.doWork(NIOServerCnxnFactory.java:522) at org.apache.zookeeper.server.WorkerService$ScheduledWorkRequest.run(WorkerService.java:154) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) org.opentest4j.AssertionFailedError: No controller broker is elected in time period at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:39) at org.junit.jupiter.api.Assertions.fail(Assertions.java:117) at kafka.server.MetadataRequestBetweenDifferentIbpTest.ensureControllerIn(MetadataRequestBetweenDifferentIbpTest.scala:78) at kafka.server.MetadataRequestBetweenDifferentIbpTest.testUnknownTopicId(MetadataRequestBetweenDifferentIbpTest.scala:62) -- 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