Paolo Patierno created KAFKA-20427:
--------------------------------------
Summary: Controllers can't form quorum on restart when hostnames
changes with dynamic quorum
Key: KAFKA-20427
URL: https://issues.apache.org/jira/browse/KAFKA-20427
Project: Kafka
Issue Type: Bug
Reporter: Paolo Patierno
It seems that when using the dynamic quorum, it is not possible to recover a
KRaft quorum if the controllers change their hostnames due to any DNS
infrastructure changes.
Following the test that I ran.
I start my Apache Kafka cluster with 3 controllers having hostnames like
controller-0, controller-1 and controller-2. I use the /etc/hosts to have them
pointing to localhost to run all of them on my laptop. Of course, I format all
of them by using the --initial-controllers flag and providing the needed
information (including the hostnames for each controller within the list). All
the controllers have the controller.quorum.bootstrap.servers configuration
properly defined with these hostnames. The controllers start up and they form
the quorum.
I shut down all the controllers and let's assume I am making changes to the DNS
infrastructure so that the same controllers will have a different hostname,
like controller-0-new, controller-1-new and controller-2-new (I was simulating
this locally through the /etc/hosts file. Disabling the old hostnames and
enabling the new ones to point to localhost). I change the
controller.quorum.bootstrap.servers configuration to use the new hostnames.
Then I restart all the controllers again. They are not able to form the quorum
anymore.
During my investigation, the controllers are going to use the metadata topic as
the source of truth for checking what are the voters and how to form the
quorum. Of course, the metadata topic contains a VoterSet record with the
quorum having the old hostnames. So looking at the log it's clear that each
controller loads such a record and then tries to connect to the voters with the
old hostnames (which are not reachable anymore). It seems there is no mechanism
for the controller to use the new controller.quorum.bootstrap.servers with the
new hostnames to try starting a leader election for a new quorum.
{noformat}
[2026-04-08 14:21:08,500] INFO [RaftManager id=0] Reading KRaft snapshot and
log as part of the initialization (org.apache.kafka.raft.KafkaRaftClient)
[2026-04-08 14:21:08,502] INFO [RaftManager id=0] Loading snapshot
(OffsetAndEpoch[offset=0, epoch=0]) since log start offset (0) is greater than
the internal listener's next offset (-1)
(org.apache.kafka.raft.internals.KRaftControlRecordStateMachine)
[2026-04-08 14:21:08,507] INFO [RaftManager id=0] Latest kraft.version is
KRAFT_VERSION_1 at offset -1
(org.apache.kafka.raft.internals.KRaftControlRecordStateMachine)
[2026-04-08 14:21:08,508] INFO [RaftManager id=0] Latest set of voters is
VoterSet(voters={0=VoterNode(voterKey=ReplicaKey(id=0,
directoryId=cCg_WPLrSWaPUCuIhtFHiw),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-0/<unresolved>:9093}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
1=VoterNode(voterKey=ReplicaKey(id=1, directoryId=lIZM5Y0ZQ0GNrFywdVIX6w),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-1/<unresolved>:9094}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
2=VoterNode(voterKey=ReplicaKey(id=2, directoryId=CSUAsFqGSJaBcenHEyS2Yg),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-2/<unresolved>:9095}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1])})
at offset -1 (org.apache.kafka.raft.internals.KRaftControlRecordStateMachine)
[2026-04-08 14:21:08,511] INFO [RaftManager id=0] Latest kraft.version is
KRAFT_VERSION_1 at offset 1
(org.apache.kafka.raft.internals.KRaftControlRecordStateMachine)
[2026-04-08 14:21:08,512] INFO [RaftManager id=0] Latest set of voters is
VoterSet(voters={0=VoterNode(voterKey=ReplicaKey(id=0,
directoryId=cCg_WPLrSWaPUCuIhtFHiw),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-0/<unresolved>:9093}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
1=VoterNode(voterKey=ReplicaKey(id=1, directoryId=lIZM5Y0ZQ0GNrFywdVIX6w),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-1/<unresolved>:9094}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
2=VoterNode(voterKey=ReplicaKey(id=2, directoryId=CSUAsFqGSJaBcenHEyS2Yg),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-2/<unresolved>:9095}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1])})
at offset 2 (org.apache.kafka.raft.internals.KRaftControlRecordStateMachine)
[2026-04-08 14:21:08,522] INFO [RaftManager id=0] Starting voters are
VoterSet(voters={0=VoterNode(voterKey=ReplicaKey(id=0,
directoryId=cCg_WPLrSWaPUCuIhtFHiw),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-0/<unresolved>:9093}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
1=VoterNode(voterKey=ReplicaKey(id=1, directoryId=lIZM5Y0ZQ0GNrFywdVIX6w),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-1/<unresolved>:9094}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1]),
2=VoterNode(voterKey=ReplicaKey(id=2, directoryId=CSUAsFqGSJaBcenHEyS2Yg),
listeners=Endpoints(endpoints={ListenerName(CONTROLLER)=controller-2/<unresolved>:9095}),
supportedKRaftVersion=SupportedVersionRange[min_version:0, max_version:1])})
(org.apache.kafka.raft.KafkaRaftClient)
[2026-04-08 14:21:08,531] INFO [RaftManager id=0] Attempting durable transition
to UnattachedState(epoch=2, leaderId=OptionalInt.empty,
votedKey=Optional.empty, voters=[0, 1, 2], electionTimeoutMs=1059,
highWatermark=Optional.empty) from null (org.apache.kafka.raft.QuorumState)
[2026-04-08 14:21:08,542] INFO [RaftManager id=0] Completed transition to
UnattachedState(epoch=2, leaderId=OptionalInt.empty, votedKey=Optional.empty,
voters=[0, 1, 2], electionTimeoutMs=1059, highWatermark=Optional.empty) from
null (org.apache.kafka.raft.QuorumState)
{noformat}
{noformat}
[2026-04-08 14:21:13,330] WARN [RaftManager id=0] Error connecting to node
controller-2:9095 (id: 2 rack: null isFenced: false)
(org.apache.kafka.clients.NetworkClient)java.net.UnknownHostException:
controller-2: Name or service not known at
java.base/java.net.Inet6AddressImpl.lookupAllHostAddr(Native Method) at
java.base/java.net.Inet6AddressImpl.lookupAllHostAddr(Inet6AddressImpl.java:52)
at
java.base/java.net.InetAddress$PlatformResolver.lookupByName(InetAddress.java:1211)
at
java.base/java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1828)
at
java.base/java.net.InetAddress$NameServiceAddresses.get(InetAddress.java:1139)
at java.base/java.net.InetAddress.getAllByName0(InetAddress.java:1818) at
java.base/java.net.InetAddress.getAllByName(InetAddress.java:1688) at
org.apache.kafka.clients.DefaultHostResolver.resolve(DefaultHostResolver.java:27)
at org.apache.kafka.clients.ClientUtils.resolve(ClientUtils.java:125) at
org.apache.kafka.clients.ClusterConnectionStates$NodeConnectionState.resolveAddresses(ClusterConnectionStates.java:536)
at
org.apache.kafka.clients.ClusterConnectionStates$NodeConnectionState.currentAddress(ClusterConnectionStates.java:511)
at
org.apache.kafka.clients.ClusterConnectionStates.currentAddress(ClusterConnectionStates.java:173)
at
org.apache.kafka.clients.NetworkClient.initiateConnect(NetworkClient.java:1140)
at org.apache.kafka.clients.NetworkClient.ready(NetworkClient.java:368) at
org.apache.kafka.server.util.InterBrokerSendThread.sendRequests(InterBrokerSendThread.java:146)
at
org.apache.kafka.server.util.InterBrokerSendThread.pollOnce(InterBrokerSendThread.java:109)
at
org.apache.kafka.server.util.InterBrokerSendThread.doWork(InterBrokerSendThread.java:137)
at
org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:136){noformat}
This result looks like a regression compared to using the static quorum (or
ZooKeeper in the past). Doing the same test but using the
controller.quorum.voters configuration, on restart, the controllers are using
such configuration as the source of truth to reach the voters and, taking into
account the new hostnames, they are able to form the quorum again.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)