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

Edoardo Comar commented on KAFKA-14996:
---------------------------------------

I found another way to get into the error state.

3 broker/controller cluster, all 3 voters. If I shut down the 2 non-active 
quorum members, the remaining acive controller enters the state where it logs 
`[2023-05-24 16:29:45,129] WARN [BrokerToControllerChannelManager id=1 
name=heartbeat] Received error UNKNOWN_SERVER_ERROR from node 1 when making an 
ApiVersionsRequest with correlation id 3945. Disconnecting. 
(org.apache.kafka.clients.NetworkClient)`

and correspondingly 
```
[2023-05-24 16:29:45,128] WARN [QuorumController id=1] getFinalizedFeatures: 
failed with unknown server exception RuntimeException in 222 us.  The 
controller is already in standby mode. 
(org.apache.kafka.controller.QuorumController)
java.lang.RuntimeException: No in-memory snapshot for epoch 159730. Snapshot 
epochs are:
    at 
org.apache.kafka.timeline.SnapshotRegistry.getSnapshot(SnapshotRegistry.java:173)
    at 
org.apache.kafka.timeline.SnapshotRegistry.iterator(SnapshotRegistry.java:131)
    at org.apache.kafka.timeline.TimelineObject.get(TimelineObject.java:69)
    at 
org.apache.kafka.controller.FeatureControlManager.finalizedFeatures(FeatureControlManager.java:303)
    at 
org.apache.kafka.controller.QuorumController.lambda$finalizedFeatures$16(QuorumController.java:2016)
    at 
org.apache.kafka.controller.QuorumController$ControllerReadEvent.run(QuorumController.java:546)
```
in controller.log

> CreateTopic falis with UnknownServerException if num partitions >= 
> QuorumController.MAX_RECORDS_PER_BATCH 
> ----------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-14996
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14996
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>            Reporter: Edoardo Comar
>            Assignee: Edoardo Comar
>            Priority: Critical
>
> If an attempt is made to create a topic with
> num partitions >= QuorumController.MAX_RECORDS_PER_BATCH  (10000)
> the client receives an UnknownServerException - it could rather receive a 
> better error.
> The controller logs
> {{2023-05-12 19:25:10,018] WARN [QuorumController id=1] createTopics: failed 
> with unknown server exception IllegalStateException at epoch 2 in 21956 us.  
> Renouncing leadership and reverting to the last committed offset 174. 
> (org.apache.kafka.controller.QuorumController)}}
> {{java.lang.IllegalStateException: Attempted to atomically commit 10001 
> records, but maxRecordsPerBatch is 10000}}
> {{    at 
> org.apache.kafka.controller.QuorumController.appendRecords(QuorumController.java:812)}}
> {{    at 
> org.apache.kafka.controller.QuorumController$ControllerWriteEvent.run(QuorumController.java:719)}}
> {{    at 
> org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:127)}}
> {{    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:829)}}
> {{[}}



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

Reply via email to