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

Daniele Carminati commented on KAFKA-15513:
-------------------------------------------

Hi there
I made an error reporting my tests:
i wrote:
"I was able to  build a cluster  as follows:

sasL :  controller-> controller

sasl:   broker -> controller

scram:   broker-> broker

scarm:  client -> broker"


Actually i didn't manage to setup

sasl scram:   broker-> broker

but only

sasl:plain   broker ->  broker

So i assume that  the authentication method must be the same on the
controller and on the broker

and for Sasl the only possibilty is SASL/PLAIN











Il giorno mar 14 nov 2023 alle ore 20:51 Proven Provenzano (Jira) <



> KRaft cluster fails with SCRAM authentication enabled for control-plane
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-15513
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15513
>             Project: Kafka
>          Issue Type: Bug
>          Components: kraft
>    Affects Versions: 3.6.0, 3.5.1
>            Reporter: migruiz4
>            Priority: Major
>
> We have observed a scenario where a KRaft cluster fails to bootstrap when 
> using SCRAM authentication for controller-to-controller communications.
> The steps to reproduce are simple:
>  * Deploy (at least) 2 Kafka servers using latest version 3.5.1.
>  * Configure a KRaft cluster, where the controller listener uses 
> SASL_PLAINTEXT + SCRAM-SHA-256 or SCRAM-SHA-512. In my case, I'm using the 
> recommended in-line jaas config 
> '{{{}listener.name.<controller-name>.scram-sha-512.sasl.jaas.config{}}}'
>  * Run 'kafka-storage.sh' in both nodes using option '--add-scram' to create 
> the SCRAM user.
> When initialized, Controllers will fail to connect to each other with an 
> authentication error:
>  
> {code:java}
> [2023-08-01 11:12:45,295] ERROR [kafka-1-raft-outbound-request-thread]: 
> Failed to send the following request due to authentication error: 
> ClientRequest(expectResponse=true, 
> callback=kafka.raft.KafkaNetworkChannel$$Lambda$687/0x00007f27d443fc60@2aba6075,
>  destination=0, correlationId=129, clientId=raft-client-1, 
> createdTimeMs=1690888364960, 
> requestBuilder=VoteRequestData(clusterId='abcdefghijklmnopqrstug', 
> topics=[TopicData(topicName='__cluster_metadata', 
> partitions=[PartitionData(partitionIndex=0, candidateEpoch=4, candidateId=1, 
> lastOffsetEpoch=0, lastOffset=0)])])) (kafka.raft.RaftSendThread) {code}
> Some additional details about the scenario that we tested out:
>  *  Controller listener does work when configured with SASL+PLAIN
>  * The issue only affects the Controller listener, SCRAM users created using 
> the same method work for data-plane listeners and inter-broker listeners.
>  
> Below you can find the exact configuration and command used to deploy:
>  * server.properties
> {code:java}
> listeners=INTERNAL://:9092,CLIENT://:9091,CONTROLLER://:9093
> advertised.listeners=INTERNAL://kafka-0:9092,CLIENT://:9091
> listener.security.protocol.map=INTERNAL:PLAINTEXT,CLIENT:PLAINTEXT,CONTROLLER:SASL_PLAINTEXT
> num.network.threads=3
> num.io.threads=8
> socket.send.buffer.bytes=102400
> socket.receive.buffer.bytes=102400
> socket.request.max.bytes=104857600
> log.dirs=/bitnami/kafka/data
> num.partitions=1
> num.recovery.threads.per.data.dir=1
> offsets.topic.replication.factor=1
> transaction.state.log.replication.factor=1
> transaction.state.log.min.isr=1
> log.retention.hours=168
> log.retention.check.interval.ms=300000
> controller.listener.names=CONTROLLER
> controller.quorum.voters=0@kafka-0:9093,1@kafka-1:9093
> inter.broker.listener.name=INTERNAL
> node.id=0
> process.roles=controller,broker
> sasl.enabled.mechanisms=PLAIN,SCRAM-SHA-256,SCRAM-SHA-512
> sasl.mechanism.controller.protocol=SCRAM-SHA-512
> listener.name.controller.sasl.enabled.mechanisms=SCRAM-SHA-512
> listener.name.controller.scram-sha-512.sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule
>  required username="controller_user" password="controller_password";{code}
>  * kafka-storage.sh command
> {code:java}
> kafka-storage.sh format --config /path/to/server.properties 
> --ignore-formatted --cluster-id abcdefghijklmnopqrstuv --add-scram 
> SCRAM-SHA-512=[name=controller_user,password=controller_password] {code}



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

Reply via email to