Details
-
Bug
-
Status: Resolved
-
Blocker
-
Resolution: Fixed
-
3.7.0
-
None
-
None
Description
When running Kafka 3.7.0-RC2 as a KRaft cluster with metadata version set to 3.6-IV2 metadata version, it throws repeated errors like this in the controller logs:
2024-01-13 16:58:01,197 INFO [QuorumController id=0] assignReplicasToDirs: event failed with UnsupportedVersionException in 15 microseconds. (org.apache.kafka.controller.QuorumController) [quorum-controller-0-event-handler]
2024-01-13 16:58:01,197 ERROR [ControllerApis nodeId=0] Unexpected error handling request RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, apiVersion=0, clientId=1000, correlationId=14, headerVersion=2) – AssignReplicasToDirsRequestData(brokerId=1000, brokerEpoch=5, directories=[DirectoryData(id=w_uxN7pwQ6eXSMrOKceYIQ, topics=[TopicData(topicId=bvAKLSwmR7iJoKv2yZgygQ, partitions=[PartitionData(partitionIndex=2), PartitionData(partitionIndex=1)]), TopicData(topicId=uNe7f5VrQgO0zST6yH1jDQ, partitions=[PartitionData(partitionIndex=0)])])]) with context RequestContext(header=RequestHeader(apiKey=ASSIGN_REPLICAS_TO_DIRS, apiVersion=0, clientId=1000, correlationId=14, headerVersion=2), connectionId='172.16.14.219:9090-172.16.14.217:53590-7', clientAddress=/172.16.14.217, principal=User:CN=my-cluster-kafka,O=io.strimzi, listenerName=ListenerName(CONTROLPLANE-9090), securityProtocol=SSL, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.7.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@71004ad2]) (kafka.server.ControllerApis) [quorum-controller-0-event-handler]
java.util.concurrent.CompletionException: org.apache.kafka.common.errors.UnsupportedVersionException: Directory assignment is not supported yet.
at java.base/java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:332)
at java.base/java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:347)
at java.base/java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:636)
at java.base/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:510)
at java.base/java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2162)
at org.apache.kafka.controller.QuorumController$ControllerWriteEvent.complete(QuorumController.java:880)
at org.apache.kafka.controller.QuorumController$ControllerWriteEvent.handleException(QuorumController.java:871)
at org.apache.kafka.queue.KafkaEventQueue$EventContext.completeWithException(KafkaEventQueue.java:148)
at org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:137)
at org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:210)
at org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:181)
at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Directory assignment is not supported yet.
With the metadata version set to 3.6-IV2, it makes sense that the request is not supported. But the request should in such case not be sent at all.
Attachments
Issue Links
- is related to
-
KAFKA-16132 Upgrading from 3.6 to 3.7 in KRaft will have seconds of partitions unavailable
- Resolved
- links to