[ https://issues.apache.org/jira/browse/KAFKA-14449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17953813#comment-17953813 ]
Zhou Shihuan edited comment on KAFKA-14449 at 5/24/25 10:20 AM: ---------------------------------------------------------------- I ran into a similar problem—upgraded nodes started sending FETCH requests at a newer version to non-upgraded nodes, which couldn’t handle them. Error logs: [2025-05-24 10:14:42,140] ERROR Closing socket for 172.31.1.87:9090-172.31.1.186:38177-779 because of error (kafka.network.Processor) org.apache.kafka.common.errors.InvalidRequestException: Error getting request for apiKey: FETCH, apiVersion: 14, connectionId: 172.31.1.87:9090-172.31.1.186:38177-779, listenerName: ListenerName(REPLICA), principal: User:ANONYMOUS Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Can't read version 14 of FetchTopi The valid FETCH request versions of non-upgraded nods is 0-13, but the replica fetch request use ApiKeys.FETCH.latestVersion(14) {code:java} // LocalLeaderEndPoint.scala private def buildFetchForPartition(topicPartition: TopicPartition, fetchState: PartitionFetchState): ResultWithPartitions[Option[ReplicaFetch]] = { ... val fetchRequestOpt = if (requestMap.isEmpty) { None } else { val version: Short = if (fetchState.topicId.isEmpty) 12 else ApiKeys.FETCH.latestVersion ... } ResultWithPartitions(fetchRequestOpt, partitionsWithError) }{code} was (Author: JIRAUSER304125): I ran into a similar problem—upgraded nodes started sending FETCH requests at a newer version to non-upgraded nodes, which couldn’t handle them. Error logs: [2025-05-24 10:14:42,140] ERROR Closing socket for 172.31.1.87:9090-172.31.1.186:38177-779 because of error (kafka.network.Processor) org.apache.kafka.common.errors.InvalidRequestException: Error getting request for apiKey: FETCH, apiVersion: 14, connectionId: 172.31.1.87:9090-172.31.1.186:38177-779, listenerName: ListenerName(REPLICA), principal: User:ANONYMOUS Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Can't read version 14 of FetchTopi LocalLeaderEndPoint.scala: {code:java} private def buildFetchForPartition(topicPartition: TopicPartition, fetchState: PartitionFetchState): ResultWithPartitions[Option[ReplicaFetch]] = { ... val fetchRequestOpt = if (requestMap.isEmpty) { None } else { val version: Short = if (fetchState.topicId.isEmpty) 12 else ApiKeys.FETCH.latestVersion ... } ResultWithPartitions(fetchRequestOpt, partitionsWithError) }{code} Perhaps we should consider version compatibility when sending requests within the cluster. > Brokers not re-joining the ISR list and stuck at started until all the > brokers restart > -------------------------------------------------------------------------------------- > > Key: KAFKA-14449 > URL: https://issues.apache.org/jira/browse/KAFKA-14449 > Project: Kafka > Issue Type: Bug > Components: core > Affects Versions: 3.2.0 > Reporter: Swathi Mocharla > Priority: Critical > > hi, > We are upgrading a 3 broker cluster (1001,1002,1003) from 3.1.0 to 3.2.0. > During upgrade, it is noticed that when 1003 is restarted, it doesn't join > back the ISR list and the broker is stuck. Same is the case with 1002. > Only when 1001 is restrarted, 1003,1002 re-join the ISR list and start > replicating data. > > {code:java} > {"type":"log", "host":"kf-pl47-me8-2", "level":"INFO", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:07:30.386", "timezone":"UTC", "log":{"message":"main - > kafka.server.KafkaServer - [KafkaServer id=1003] started"}} > {"type":"log", "host":"kf-pl47-me8-2", "level":"INFO", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:07:30.442", "timezone":"UTC", > "log":{"message":"data-plane-kafka-request-handler-1 - state.change.logger - > [Broker id=1003] Add 397 partitions and deleted 0 partitions from metadata > cache in response to UpdateMetadata request sent by controller 1002 epoch 18 > with correlation id 0"}} > {"type":"log", "host":"kf-pl47-me8-2", "level":"INFO", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:07:30.448", "timezone":"UTC", > "log":{"message":"BrokerToControllerChannelManager broker=1003 name=alterIsr > - kafka.server.BrokerToControllerRequestThread - > [BrokerToControllerChannelManager broker=1003 name=alterIsr]: Recorded new > controller, from now on will use broker > kf-pl47-me8-1.kf-pl47-me8-headless.nc0968-admin-ns.svc.cluster.local:9092 > (id: 1002 rack: null)"}} > {"type":"log", "host":"kf-pl47-me8-2", "level":"ERROR", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:07:30.451", "timezone":"UTC", > "log":{"message":"data-plane-kafka-network-thread-1003-ListenerName(PLAINTEXT)-PLAINTEXT-1 > - kafka.network.Processor - Closing socket for > 192.168.216.11:9092-192.168.199.100:53778-0 because of error"}} > org.apache.kafka.common.errors.InvalidRequestException: Error getting request > for apiKey: LEADER_AND_ISR, apiVersion: 6, connectionId: > 192.168.216.11:9092-192.168.199.100:53778-0, listenerName: > ListenerName(PLAINTEXT), principal: User:ANONYMOUS > org.apache.kafka.common.errors.InvalidRequestException: Error getting request > for apiKey: LEADER_AND_ISR, apiVersion: 6, connectionId: > 192.168.216.11:9092-192.168.235.153:46282-461, listenerName: > ListenerName(PLAINTEXT), principal: User:ANONYMOUS > Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Can't > read version 6 of LeaderAndIsrTopicState > {"type":"log", "host":"kf-pl47-me8-2", "level":"INFO", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:12:50.916", "timezone":"UTC", > "log":{"message":"controller-event-thread - kafka.controller.KafkaController > - [Controller id=1003] 1003 successfully elected as the controller. Epoch > incremented to 20 and epoch zk version is now 20"}} > {"type":"log", "host":"kf-pl47-me8-2", "level":"INFO", > "neid":"kafka-b352b4f8cf4447e9a73d9e7ef3ec746c", "system":"kafka", > "time":"2022-12-06T10:12:50.917", "timezone":"UTC", > "log":{"message":"controller-event-thread - kafka.controller.KafkaController > - [Controller id=1003] Registering handlers"}} > {code} > Note: Unclean leader election is not enabled. > This possibly was introduced by KAFKA-13587. > In the below snapshot during the upgrade, at 16:05:15 UTC 2022, 1001 was > restarting and both 1002 and 1003 were already up and running (after the > upgrade from 3.1.0 to 3.2.0), but did not manage to re-join the ISRs. > {code:java} > Wed Dec 7 16:05:15 UTC 2022 > Topic: test TopicId: L6Yj_Nf9RrirNhFQzvXODw PartitionCount: 2 > ReplicationFactor: 3 Configs: > compression.type=producer,min.insync.replicas=1,cleanup.policy=delete,flush.ms=1000,segment.bytes=100000000,flush.messages=10000,max.message.bytes=1000012,index.interval.bytes=4096,unclean.leader.election.enable=false,retention.bytes=1000000000,segment.index.bytes=10485760 > Topic: test Partition: 0 Leader: none Replicas: > 1002,1003,1001 Isr: 1001 > Topic: test Partition: 1 Leader: none Replicas: > 1001,1002,1003 Isr: 1001 > Wed Dec 7 16:05:33 UTC 2022 > Topic: test TopicId: L6Yj_Nf9RrirNhFQzvXODw PartitionCount: 2 > ReplicationFactor: 3 Configs: > compression.type=producer,min.insync.replicas=1,cleanup.policy=delete,flush.ms=1000,segment.bytes=100000000,flush.messages=10000,max.message.bytes=1000012,index.interval.bytes=4096,unclean.leader.election.enable=false,retention.bytes=1000000000,segment.index.bytes=10485760 > Topic: test Partition: 0 Leader: 1001 Replicas: > 1002,1003,1001 Isr: 1001,1002,1003 > Topic: test Partition: 1 Leader: 1001 Replicas: > 1001,1002,1003 Isr: 1001,1002,1003{code} > Is there anything the user needs to do explicitly to work around this issue? -- This message was sent by Atlassian Jira (v8.20.10#820010)