[ 
https://issues.apache.org/jira/browse/KAFKA-16163?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17808336#comment-17808336
 ] 

Mickael Maison commented on KAFKA-16163:
----------------------------------------

It looks like this behavior was introduced in 
https://github.com/apache/kafka/commit/37416e1aebae33d01d5059ba906ec8e0e1107284

> Constant resignation/reelection of controller when starting a single node in 
> combined mode
> ------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-16163
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16163
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 3.7.0
>            Reporter: Mickael Maison
>            Priority: Major
>
> When starting a single node in combined mode:
> {noformat}
> $ KAFKA_CLUSTER_ID="$(bin/kafka-storage.sh random-uuid)"
> $ bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c 
> config/kraft/server.properties
> $ bin/kafka-server-start.sh config/kraft/server.properties{noformat}
>  
> it's constantly spamming the logs with:
> {noformat}
> [2024-01-18 17:37:09,065] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:11,967] INFO [RaftManager id=1] Did not receive fetch 
> request from the majority of the voters within 3000ms. Current fetched voters 
> are []. (org.apache.kafka.raft.LeaderState)
> [2024-01-18 17:37:11,967] INFO [RaftManager id=1] Completed transition to 
> ResignedState(localId=1, epoch=138, voters=[1], electionTimeoutMs=1864, 
> unackedVoters=[], preferredSuccessors=[]) from Leader(localId=1, epoch=138, 
> epochStartOffset=829, highWatermark=Optional[LogOffsetMetadata(offset=835, 
> metadata=Optional[(segmentBaseOffset=0,relativePositionInSegment=62788)])], 
> voterStates={1=ReplicaState(nodeId=1, 
> endOffset=Optional[LogOffsetMetadata(offset=835, 
> metadata=Optional[(segmentBaseOffset=0,relativePositionInSegment=62788)])], 
> lastFetchTimestamp=-1, lastCaughtUpTimestamp=-1, 
> hasAcknowledgedLeader=true)}) (org.apache.kafka.raft.QuorumState)
> [2024-01-18 17:37:13,072] INFO [NodeToControllerChannelManager id=1 
> name=heartbeat] Client requested disconnect from node 1 
> (org.apache.kafka.clients.NetworkClient)
> [2024-01-18 17:37:13,072] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,123] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,124] INFO [NodeToControllerChannelManager id=1 
> name=heartbeat] Client requested disconnect from node 1 
> (org.apache.kafka.clients.NetworkClient)
> [2024-01-18 17:37:13,124] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,175] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,176] INFO [NodeToControllerChannelManager id=1 
> name=heartbeat] Client requested disconnect from node 1 
> (org.apache.kafka.clients.NetworkClient)
> [2024-01-18 17:37:13,176] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,227] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,229] INFO [NodeToControllerChannelManager id=1 
> name=heartbeat] Client requested disconnect from node 1 
> (org.apache.kafka.clients.NetworkClient)
> [2024-01-18 17:37:13,229] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread)
> [2024-01-18 17:37:13,279] INFO 
> [broker-1-to-controller-heartbeat-channel-manager]: Recorded new controller, 
> from now on will use node localhost:9093 (id: 1 rack: null) 
> (kafka.server.NodeToControllerRequestThread){noformat}
> This did not happen in 3.6.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to