You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Igor Shipenkov (Jira)" <ji...@apache.org> on 2021/11/24 03:33:00 UTC

[jira] [Created] (KAFKA-13474) Regression in dynamic update of client-side SSL factory

Igor Shipenkov created KAFKA-13474:
--------------------------------------

             Summary: Regression in dynamic update of client-side SSL factory
                 Key: KAFKA-13474
                 URL: https://issues.apache.org/jira/browse/KAFKA-13474
             Project: Kafka
          Issue Type: Bug
          Components: core
    Affects Versions: 2.7.2, 2.7.0
            Reporter: Igor Shipenkov
         Attachments: failed-controller-single-session-20211119.pcap.gz

h1. Problem
It seems, after updating listener SSL certificate with dynamic configuration update, old certificate is somehow still used for client SSL factory. Because of this broker fails to create new connection to controller after old certificate expires.

h1. History
Back in KAFKA-8336 there was an issue, when client-side SSL factory wasn't updating cetificate, when it was changed with dynamic configuration. That bug have been fixed in version 2.3 and I can confirm, that dynamic update worked for us with kafka 2.4. But now we have updated clusters to 2.7 and see this (or at least similar) problem again.

h1. Affected versions
First we've seen this on confluent 6.1.2, which (I think) based on kafka 2.7.0. Then I tried vanilla versions 2.7.0 and 2.7.2 and can reproduce problem on them just fine

h1. How to reproduce
* Have zookeeper somewhere (in my example it will be "10.88.0.21:2181").
* Get vanilla version 2.7.2 (or 2.7.0) from https://kafka.apache.org/downloads .
* Make basic broker config like this (don't forget to actually create log.dirs):
{code}
broker.id=1

listeners=SSL://:9092
advertised.listeners=SSL://localhost:9092

log.dirs=/tmp/broker1/data

zookeeper.connect=10.88.0.21:2181

security.inter.broker.protocol=SSL
ssl.protocol=TLSv1.2
ssl.client.auth=required
ssl.endpoint.identification.algorithm=
ssl.keystore.type=PKCS12
ssl.keystore.location=/tmp/broker1/secrets/broker1.keystore.p12
ssl.keystore.password=changeme1
ssl.key.password=changeme1
ssl.truststore.type=PKCS12
ssl.truststore.location=/tmp/broker1/secrets/truststore.p12
ssl.truststore.password=changeme
{code}
(I use here TLS 1.2 just so I can see client certificate in TLS handshake, you will get same error with default TLS 1.3 too)
** Repeat this config for another 2 brokers, changing id, listener port and certificate accordingly.
* Make basic client config (I use for it one of brokers' certificate):
{code}
security.protocol=SSL
ssl.key.password=changeme1
ssl.keystore.type=PKCS12
ssl.keystore.location=/tmp/broker1/secrets/broker1.keystore.p12
ssl.keystore.password=changeme1
ssl.truststore.type=PKCS12
ssl.truststore.location=/tmp/broker1/secrets/truststore.p12
ssl.truststore.password=changeme
ssl.endpoint.identification.algorithm=
{code}
* Create usual local self-signed PKI for test
** generate self-signed CA certificate and private key. Place certificate in truststore.
** create keys for broker certificates and create requests from them as usual (I'll use here same subject for all brokers)
** create 2 certificates as usual
{code}
openssl x509 \
       -req -CAcreateserial -days 1 \
       -CA ca/ca-cert.pem -CAkey ca/ca-key.pem \
       -in broker1.csr -out broker1.crt
{code}
** Use "faketime" utility to make third certificate expire soon:
{code}
# date here is some point yesterday, so certificate will expire like 10-15 minutes from now
faketime "2021-11-23 10:15" openssl x509 \
       -req -CAcreateserial -days 1 \
       -CA ca/ca-cert.pem -CAkey ca/ca-key.pem \
       -in broker2.csr -out broker2.crt
{code}
** create keystores from certificates and place them according to broker configs from earlier
* Run 3 brokers with your configs like
{code}
./bin/kafka-server-start.sh server2.properties
{code}
(I start it here without daemon mode to see logs right on terminal - just use "tmux" or something for run 3 brokers simultaneously)
** you can check that one broker certificate will expire soon with
{code}
openssl s_client -connect localhost:9093 </dev/null | openssl x509 -noout -text | grep -A2 Valid
{code}
* Issue new certificate to replace one, which will expire soon, place it in keystore and replace old keystore with it.
* Use dynamic configuration to make broker re-read keystore:
{code}
./bin/kafka-configs --command-config ssl.properties --bootstrap-server localhost:9092 --entity-type brokers --entity-name "2" --alter --add-config "listener.name.SSL.ssl.keystore.location=/tmp/broker2/secrets/broker2.keystore.p12"
{code}
** You can check that broker now has new certificate on its listener with same
{code}
openssl s_client -connect localhost:9093 </dev/null | openssl x509 -noout -text | grep -A2 Valid
{code}
* Wait until that old certificate expires and make some changes, which provoke broker to make new controller connection. For example if I have controller on broker "1" and expired certificate was on broker "2", then I restart broker "3".
* On broker with expired certificate you will see in log something like
{code}
INFO [broker-2-to-controller-send-thread]: Recorded new controller, from now on will use broker 1 (kafka.server.BrokerToControllerRequestThread)
INFO [broker-2-to-controller] Failed authentication with localhost/127.0.0.1 (SSL handshake failed) (org.apache.kafka.common.network.Selector)
ERROR [broker-2-to-controller] Connection to node 1 (localhost/127.0.0.1:9092) failed authentication due to: SSL handshake failed (org.apache.kafka.clients.NetworkClient)
ERROR [broker-2-to-controller-send-thread]: Failed to send the following request due to authentication error: ClientRequest(expectResponse=true, callback=kafka.server.BrokerToControllerRequestThread$$Lambda$996/0x0000000801724c40@4d3e77ce, destination=1, correlationId=626, clientId=2, createdTimeMs=1637718291682, requestBuilder=AlterIsrRequestData(brokerId=2, brokerEpoch=293, topics=<some topic topology> kafka.server.BrokerToControllerRequestThread)
{code}
and controller log will show something like
{code}
INFO [SocketServer brokerId=1] Failed authentication with /127.0.0.1 (SSL handshake failed) (org.apache.kafka.common.network.Selector)
{code}
and if broker with expired and changed certificate was controller itself it eve could not connect to itself.
* If you make traffic dump (and you use TLS 1.2 or less) then you will see that client tries to use old certificate.

Here is example of traffic dump, when broker with expired and dynamically changed certificate is current controller, so it can't connect to itself:
In this example you will see that "Server" use new certificate and "Client" use old certificate, but it's same broker!



--
This message was sent by Atlassian Jira
(v8.20.1#820001)