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

Sven Dewit commented on KAFKA-20109:
------------------------------------

[~harmadasg] thanks for your analysis. My guess so far was that the issue lies 
within the "asymmetrical" nature of the outcome of the misconfigured 
controller: while the controller cannot authorize incoming connections from 
other nodes, the other nodes will happily accept connections from the 
misconfigured controller - after all, their configuration is correct.

{quote}
On the other hand it is not exactly desirable that a single misconfigured 
controller can cause a cascading cluster failure. Hard to say how we can 
improve the current experience. Maybe we can call out the importance of the 
principal name related configs in the KRaft documentation 
(ssl.principal.mapping.rules, kerberos.principal.to.local.rules, 
principal.builder.class).
{quote}

That's exactly what I think is important here. The importance of correct 
configuration of the principal mapping rules is crystal clear (to me, at 
least). Having a wrong configuration on a single controller should not lead to 
such a disastrous outcome, though. After all, as it is right now, there's no 
chance to spot the erroneous configuration on a cluster this way other than 
having the cluster die.

Btw., the issue still occurs on Kafka *4.2.0*. But: the point in time when it 
occurs has changed slightly. While on Kafka <4.2.0, the cluster died virtually 
instantly (i.e. a couple of seconds) after the misconfigured controller has 
been started up. With 4.2.0, it seems more random when the cluster dies, in 
most of my tests I had to restart at least one of the other controllers to have 
the failure happen.

> Complete Kafka cluster dies on incorrect SSL config of a single controller
> --------------------------------------------------------------------------
>
>                 Key: KAFKA-20109
>                 URL: https://issues.apache.org/jira/browse/KAFKA-20109
>             Project: Kafka
>          Issue Type: Bug
>          Components: config, controller
>    Affects Versions: 4.1.1
>         Environment: Debian trixie x86_64, Apache Kafka 3.9.0 - 4.1.1
>            Reporter: Sven Dewit
>            Assignee: Gergely Harmadás
>            Priority: Major
>         Attachments: controller3.log, reproduce.tar.gz
>
>
> Hello,
> we've recently run into a bug in Apache Kafka in Kraft mode where a whole 
> mtls-enabled cluster (controllers + brokers) die if a single controller is 
> (re)started with bad ssl principal mapping rules.
> The bad config of course was appllied unintentionally when doing some changes 
> in the config management of the system, basically it led to 
> {{ssl.principal.mapping.rules}} missing for the controller listener on that 
> one node. As soon as this single controller was restarted, the whole cluster 
> died within seconds, both controllers and brokers, with this error message:
> {code:java}
> ERROR Encountered fatal fault: Unexpected error in raft IO thread 
> (org.apache.kafka.server.fault.ProcessTerminatingFaultHandler)
> org.apache.kafka.common.errors.ClusterAuthorizationException: Received 
> cluster authorization error in response InboundResponse(correlationId=493, 
> data=BeginQuorumEpochResponseData(errorCode=31, topics=[], nodeEndpoints=[]), 
> source=controller-3:9093 (id: 103 rack: null isFenced: false)) {code}
> While the missing/bad ssl principal mapping is a major misconfiguration on a 
> cluster where in-cluster communication is based on mtls, this still should 
> not lead to the whole cluster terminating.
> The issue occurred on version 4.1.1 of Apache Kafka, but could be reproduced 
> back to 3.9.0.
> To reproduce, see the attached tarball containing
>  * {{gen-test-ca-and-certs.sh}} to create ca and certificates for brokers and 
> controllers to work in mtls mode
>  * {{compose.yml}} to spin up the cluster with {{podman compose}}
> Once the cluster is running, the following steps reproduce the error:
>  * {{podman compose down controller-3}} to stop controller 3
>  * uncomment line 53 in {{compose.yml}} to delete controller 3's 
> {{ssl.principal.mapping.rules}}
>  * {{podman compose up controller-3}} and watch the cluster go down the drain
>  
> In case I can provide you with any more information or support don't hesitate 
> to reach out to me.
>  
> Best regards,
> Sven



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

Reply via email to