[
https://issues.apache.org/jira/browse/KAFKA-20109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18061666#comment-18061666
]
Gergely Harmadás commented on KAFKA-20109:
------------------------------------------
Hi [~svdewitmam], here is my analysis:
Before going into the details note that
* misconfigured {{ssl.principal.mapping.rules}} config means that a given
broker/controller is not able to properly extract the principal names of the
other brokers/controllers from the incoming requests
* since not the desired principal names are extracted all the requests fail
with authorization exception which are handled by the misconfigured
broker/controller
* for the thread handling KRaft related requests receiving a
{{ClusterAuthorizationException}} is fatal, meaning that it causes the
broker/controller process to shut down immediately
The timeline for the scenario described by you
# Initial state is that the cluster is working as expected,
{{ssl.principal.mapping.rules}} config is correct on every node
** principal name for the controllers: *kafka-controller*
** principal name for the brokers: *kafka-broker*
** corresponding config {{super.users=User:kafka-controller;User:kafka-broker}}
# Controller 3 is shut down and restarted with
{{{}listener.name.controller.ssl.principal.mapping.rules=null{}}}, which
basically means that use the default value for {{ssl.principal.mapping.rules}}
# Controller 3 now extracts principal names from incoming requests by taking
the distinguished name of the X.500 certificate as dictated by the [default
config
|https://kafka.apache.org/41/configuration/broker-configs/#brokerconfigs_ssl.principal.mapping.rules]
** e.g. in your case a possible resulting principal name is
{{O=Test-Org,OU=Dev,CN=controller-2}}
# When Controller 3 is online again it receives a {{BEGIN_QUORUM_EPOCH}}
request from the currently active controller
** the {{BEGIN_QUORUM_EPOCH}} request acts as a heartbeat from the active
controller which is sent continuously to every member of the voter set, see
KAFKA-16536
# {*}Now the crucial point{*}. Controller 3 is not able to properly extract
the principal name of the currently active controller as *kafka-controller*
which means that the {{ClusterAuthorizationException}} is returned in the
response
# As mentioned above receiving a {{ClusterAuthorizationException}} is fatal
and the currently active controller crashes
# After the crash the other controllers/brokers try to fetch from a random
controller in order to discover the new active controller. Controller 3 is a
valid target for these fetch requests
# Eventually Controller 3 will receive {{FETCH}} requests from the other
brokers/controller (including itself) and will cause a similar fatal
ClusterAuthorizationException response which crashes all the other nodes
Also updated logs for Controller 3 which makes it easier to replay the events
I am not an expert on how KRaft operates, but to me this looks like a valid
response to a serious misconfiguration. Authorization failed responses must be
taken seriously in order to maintain the integrity of the cluster. Additionally
the issue is not specific to mTLS, a similar thing can happen also on
SASL/GSSAPI with a misconfigured {{sasl.kerberos.principal.to.local.rules}}
config and both mTLS and SASL/GSSAPI can be affected by an issue in a custom
{{{}principal.builder.class{}}}.
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).
[~kevinwu2412] [~jsancio] apologies for the ping, I hope I correctly identified
you as experts in KRaft related things based on the commit history. If you have
some bandwidth please share your opinion about the issue and the proposed
improvement in the KRaft docs.
> 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)