You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Florian Blumenstein <fl...@bbraun.com> on 2022/05/16 11:42:26 UTC

Switchin from Zookepper to Kafka KRaft mode / Using ACLs with Kafka KRaft mode

Hi guys,

I currently try to switch from Kafka 3.1.0 with ZooKeeper to Kafka 3.2.0 with Kafka Kraft mode. I adjusted the server.properties as follows:

### KRaft-properties
process.roles=broker,controller
node.id=1
controller.quorum.voters=1@127.0.0.1:9091
controller.listener.names=CONTROLLER

auto.create.topics.enable=false
ssl.client.auth=required

### Enable ACLs
authorizer.class.name=org.apache.kafka.metadata.authorizer.StandardAuthorizer
allow.everyone.if.no.acl.found=false

# Topics and indexes are stored here to keep track of records sent via broker
log.dir=/opt/kafka/data/

############################# Internal Topic Settings  #############################
# The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state"
# For anything other than development testing, a value greater than 1 is recommended for to ensure availability such as 3.
offsets.topic.replication.factor=1
transaction.state.log.replication.factor=1
transaction.state.log.min.isr=1

### Platform Configured Entries --- Below here entries are configured by the platform
listener.name.docker.ssl.keystore.location=/app/ssl/internalKeystore.jks
super.users=User:Applications:0765df41-0b31-4db8-8849-c9d77e9c6e20;User:CN=onlinesuiteplus-kafka,OU=Services,O=Company AG,L=City,C=DE
advertised.listeners=DEVIN://onlinesuiteplus-kafka:29092,DEVOUT://localhost:9092,DOCKER://onlinesuiteplus-kafka:29093,EXTERNAL://localhost:9093
listener.name.docker.ssl.key.password=password
inter.broker.listener.name=DOCKER
listener.name.external.ssl.key.password=password
listener.name.external.ssl.truststore.password=password
ssl.principal.mapping.rules=RULE:^CN=(.*?),OU=Applications.*$/Applications:$1/,RULE:^CN=(.*?),OU=Devices.*$/Devices:$1/,DEFAULT
initial.start=true
listener.name.docker.ssl.truststore.location=/app/ssl/truststore.jks
listener.name.external.ssl.keystore.password=password
listeners=CONTROLLER://:9091,DEVIN://:29092,DEVOUT://:9092,DOCKER://:29093,EXTERNAL://:9093
listener.name.external.ssl.truststore.location=/app/ssl/truststore.jks
listener.name.docker.ssl.truststore.password=password
listener.name.external.ssl.keystore.location=/app/ssl/externalKeystore.jks
listener.security.protocol.map=CONTROLLER:PLAINTEXT,DEVIN:PLAINTEXT,DEVOUT:PLAINTEXT,DOCKER:SSL,EXTERNAL:SSL
listener.name.docker.ssl.keystore.password=password

If I now run kafka with the following script:

if [ "$KAFKA_INITIAL_START" == "true" ]
then
    echo "Running kafka-storage.sh because env var KAFKA_INITIAL_START was set to true"
    "${KAFKA_HOME}"/bin/kafka-storage.sh format --config "${KAFKA_HOME}"/config/server.properties --cluster-id $("${KAFKA_HOME}"/bin/kafka-storage.sh random-uuid)
fi

exec "$KAFKA_HOME/bin/kafka-server-start.sh" "$KAFKA_HOME/config/server.properties"


I got the following logs:

[2022-05-16 11:25:08,894] INFO Registered kafka:type=kafka.Log4jController MBean (kafka.utils.Log4jControllerRegistration$)
[2022-05-16 11:25:09,220] INFO Setting -D jdk.tls.rejectClientInitiatedRenegotiation=true to disable client-initiated TLS renegotiation (org.apache.zookeeper.common.X509Util)
[2022-05-16 11:25:09,473] INFO [LogLoader partition=__cluster_metadata-0, dir=/opt/kafka/data] Loading producer state till offset 0 with message format version 2 (kafka.log.UnifiedLog$)
[2022-05-16 11:25:09,474] INFO [LogLoader partition=__cluster_metadata-0, dir=/opt/kafka/data] Reloading from producer snapshot and rebuilding producer state from offset 0 (kafka.log.UnifiedLog$)
[2022-05-16 11:25:09,477] INFO [LogLoader partition=__cluster_metadata-0, dir=/opt/kafka/data] Producer state recovery took 2ms for snapshot load and 0ms for segment recovery from offset 0 (kafka.log.UnifiedLog$)
[2022-05-16 11:25:09,584] INFO [raft-expiration-reaper]: Starting (kafka.raft.TimingWheelExpirationService$ExpiredOperationReaper)
[2022-05-16 11:25:09,784] INFO [RaftManager nodeId=1] Completed transition to Unattached(epoch=0, voters=[1], electionTimeoutMs=1442) (org.apache.kafka.raft.QuorumState)
[2022-05-16 11:25:09,797] INFO [RaftManager nodeId=1] Completed transition to CandidateState(localId=1, epoch=1, retries=1, electionTimeoutMs=1741) (org.apache.kafka.raft.QuorumState)
[2022-05-16 11:25:09,810] INFO [RaftManager nodeId=1] Completed transition to Leader(localId=1, epoch=1, epochStartOffset=0, highWatermark=Optional.empty, voterStates={1=ReplicaState(nodeId=1, endOffset=Optional.empty, lastFetchTimestamp=OptionalLong.empty, hasAcknowledgedLeader=true)}) (org.apache.kafka.raft.QuorumState)
[2022-05-16 11:25:09,854] INFO Registered signal handlers for TERM, INT, HUP (org.apache.kafka.common.utils.LoggingSignalHandler)
[2022-05-16 11:25:09,860] INFO [kafka-raft-outbound-request-thread]: Starting (kafka.raft.RaftSendThread)
[2022-05-16 11:25:09,860] INFO [kafka-raft-io-thread]: Starting (kafka.raft.KafkaRaftManager$RaftIoThread)
[2022-05-16 11:25:09,862] INFO Starting controller (kafka.server.ControllerServer)
[2022-05-16 11:25:09,869] INFO [StandardAuthorizer 1] set super.users=User:CN=onlinesuiteplus-kafka,OU=Services,O=B. Braun Melsungen AG,L=Melsungen,C=DE,User:Applications:0765df41-0b31-4db8-8849-c9d77e9c6e20, default result=DENIED (org.apache.kafka.metadata.authorizer.StandardAuthorizerData)
[2022-05-16 11:25:10,270] INFO Updated connection-accept-rate max connection creation rate to 2147483647 (kafka.network.ConnectionQuotas)
[2022-05-16 11:25:10,277] INFO Awaiting socket connections on 0.0.0.0:9091. (kafka.network.DataPlaneAcceptor)
[2022-05-16 11:25:10,318] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Created data-plane acceptor and processors for endpoint : ListenerName(CONTROLLER) (kafka.network.SocketServer)
[2022-05-16 11:25:10,354] INFO [RaftManager nodeId=1] Registered the listener org.apache.kafka.controller.QuorumController$QuorumMetaLogListener@557176505 (org.apache.kafka.raft.KafkaRaftClient)
[2022-05-16 11:25:10,361] INFO [ThrottledChannelReaper-Fetch]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,362] INFO [ThrottledChannelReaper-Produce]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,363] INFO [ThrottledChannelReaper-Request]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,365] INFO [ThrottledChannelReaper-ControllerMutation]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,366] INFO [Controller 1] Becoming the active controller at epoch 1, committed offset -1 and committed epoch -1. (org.apache.kafka.controller.QuorumController)
[2022-05-16 11:25:10,384] INFO [ExpirationReaper-1-AlterAcls]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:10,402] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Starting socket server acceptors and processors (kafka.network.SocketServer)
[2022-05-16 11:25:10,434] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Started data-plane acceptor and processor(s) for endpoint : ListenerName(CONTROLLER) (kafka.network.SocketServer)
[2022-05-16 11:25:10,435] INFO [SocketServer listenerType=CONTROLLER, nodeId=1] Started socket server acceptors and processors (kafka.network.SocketServer)
[2022-05-16 11:25:10,436] INFO [BrokerServer id=1] Transition from SHUTDOWN to STARTING (kafka.server.BrokerServer)
[2022-05-16 11:25:10,437] INFO [BrokerServer id=1] Starting broker (kafka.server.BrokerServer)
[2022-05-16 11:25:10,457] INFO [ThrottledChannelReaper-Fetch]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,457] INFO [ThrottledChannelReaper-Produce]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,458] INFO [ThrottledChannelReaper-Request]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,458] INFO [ThrottledChannelReaper-ControllerMutation]: Starting (kafka.server.ClientQuotaManager$ThrottledChannelReaper)
[2022-05-16 11:25:10,491] INFO [BrokerToControllerChannelManager broker=1 name=forwarding]: Starting (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:10,492] INFO [BrokerToControllerChannelManager broker=1 name=forwarding]: Recorded new controller, from now on will use broker localhost:9091 (id: 1 rack: null) (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:10,552] INFO Updated connection-accept-rate max connection creation rate to 2147483647 (kafka.network.ConnectionQuotas)
[2022-05-16 11:25:10,553] INFO Awaiting socket connections on 0.0.0.0:29092. (kafka.network.DataPlaneAcceptor)
[2022-05-16 11:25:10,568] INFO [SocketServer listenerType=BROKER, nodeId=1] Created data-plane acceptor and processors for endpoint : ListenerName(DEVIN) (kafka.network.SocketServer)
[2022-05-16 11:25:10,570] INFO Updated connection-accept-rate max connection creation rate to 2147483647 (kafka.network.ConnectionQuotas)
[2022-05-16 11:25:10,571] INFO Awaiting socket connections on 0.0.0.0:9092. (kafka.network.DataPlaneAcceptor)
[2022-05-16 11:25:10,583] INFO [SocketServer listenerType=BROKER, nodeId=1] Created data-plane acceptor and processors for endpoint : ListenerName(DEVOUT) (kafka.network.SocketServer)
[2022-05-16 11:25:10,585] INFO Updated connection-accept-rate max connection creation rate to 2147483647 (kafka.network.ConnectionQuotas)
[2022-05-16 11:25:10,586] INFO Awaiting socket connections on 0.0.0.0:29093. (kafka.network.DataPlaneAcceptor)
[2022-05-16 11:25:11,323] INFO [SocketServer listenerType=BROKER, nodeId=1] Created data-plane acceptor and processors for endpoint : ListenerName(DOCKER) (kafka.network.SocketServer)
[2022-05-16 11:25:11,324] INFO Updated connection-accept-rate max connection creation rate to 2147483647 (kafka.network.ConnectionQuotas)
[2022-05-16 11:25:11,325] INFO Awaiting socket connections on 0.0.0.0:9093. (kafka.network.DataPlaneAcceptor)
[2022-05-16 11:25:11,343] INFO [SocketServer listenerType=BROKER, nodeId=1] Created data-plane acceptor and processors for endpoint : ListenerName(EXTERNAL) (kafka.network.SocketServer)
[2022-05-16 11:25:11,351] INFO [BrokerToControllerChannelManager broker=1 name=alterIsr]: Starting (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:11,351] INFO [BrokerToControllerChannelManager broker=1 name=alterIsr]: Recorded new controller, from now on will use broker localhost:9091 (id: 1 rack: null) (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:11,369] INFO [ExpirationReaper-1-Produce]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,371] INFO [ExpirationReaper-1-Fetch]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,372] INFO [ExpirationReaper-1-DeleteRecords]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,374] INFO [ExpirationReaper-1-ElectLeader]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,397] INFO [ExpirationReaper-1-Heartbeat]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,398] INFO [ExpirationReaper-1-Rebalance]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,457] INFO [RaftManager nodeId=1] Registered the listener kafka.server.metadata.BrokerMetadataListener@225769393 (org.apache.kafka.raft.KafkaRaftClient)
[2022-05-16 11:25:11,457] INFO [BrokerToControllerChannelManager broker=1 name=heartbeat]: Starting (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:11,458] INFO [BrokerToControllerChannelManager broker=1 name=heartbeat]: Recorded new controller, from now on will use broker localhost:9091 (id: 1 rack: null) (kafka.server.BrokerToControllerRequestThread)
[2022-05-16 11:25:11,459] INFO [StandardAuthorizer 1] set super.users=User:CN=onlinesuiteplus-kafka,OU=Services,O=B. Braun Melsungen AG,L=Melsungen,C=DE,User:Applications:0765df41-0b31-4db8-8849-c9d77e9c6e20, default result=DENIED (org.apache.kafka.metadata.authorizer.StandardAuthorizerData)
[2022-05-16 11:25:11,464] INFO [BrokerLifecycleManager id=1] Incarnation WvNL61avTOC-nYrzNqPy6A of broker 1 in cluster 5vz8gUXVSke--ryOTMTNLg is now STARTING. (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:11,540] INFO [ExpirationReaper-1-AlterAcls]: Starting (kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2022-05-16 11:25:11,612] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=0) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=0), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:11,619] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:11,719] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=2) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=2), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:11,720] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:11,922] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=3) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=3), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:11,924] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:12,330] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=4) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=4), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:12,333] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:13,131] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=5) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=5), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(C[2022-05-16 11:25:13,133] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:14,733] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=6) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=6), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:14,734] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:17,892] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=7) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=7), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:17,894] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)
[2022-05-16 11:25:24,216] ERROR Unexpected error handling request RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=8) -- BrokerRegistrationRequestData(brokerId=1, clusterId='5vz8gUXVSke--ryOTMTNLg', incarnationId=WvNL61avTOC-nYrzNqPy6A, listeners=[Listener(name='DEVIN', host='onlinesuiteplus-kafka', port=29092, securityProtocol=0), Listener(name='DEVOUT', host='localhost', port=9092, securityProtocol=0), Listener(name='DOCKER', host='onlinesuiteplus-kafka', port=29093, securityProtocol=1), Listener(name='EXTERNAL', host='localhost', port=9093, securityProtocol=1)], features=[], rack=null) with context RequestContext(header=RequestHeader(apiKey=BROKER_REGISTRATION, apiVersion=0, clientId=1, correlationId=8), connectionId='127.0.0.1:9091-127.0.0.1:33790-0', clientAddress=/127.0.0.1, principal=User:ANONYMOUS, listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, clientInformation=ClientInformation(softwareName=apache-kafka-java, softwareVersion=3.2.0), fromPrivilegedListener=false, principalSerde=Optional[org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder@797e4004]) (kafka.server.ControllerApis)
org.apache.kafka.common.errors.ClusterAuthorizationException: Request Request(processor=0, connectionId=127.0.0.1:9091-127.0.0.1:33790-0, session=Session(User:ANONYMOUS,/127.0.0.1), listenerName=ListenerName(CONTROLLER), securityProtocol=PLAINTEXT, buffer=null, envelope=None) is not authorized.
[2022-05-16 11:25:24,218] INFO [BrokerLifecycleManager id=1] Unable to register broker 1 because the controller returned error CLUSTER_AUTHORIZATION_FAILED (kafka.server.BrokerLifecycleManager)


I understand it like this, that if I use PLAINTEXT as CONTROLLER security map entry it won't use authorization at all for the communication. I also tried to use SSL for the CONTROLLER security map entry but then I got a SSL_HANDSHAKE_FAILED error message. So what do I have to do to run Kafka in KRaft mode with ACLs enabled?

Best regards,
Florian

B. Braun Avitum AG

Vorstand:
Anna Maria Braun (Vorsitzende)
Michael Becker
Dr. Holger Seeberg

Vorsitz des Aufsichtsrats:
Benjamin Kuhnsch (stellv. Vorsitzender)

Sitz der Gesellschaft: Melsungen
Reg. Gericht: Amtsgericht Fritzlar HRB 11263

Informationen zur EU-Datenschutzgrundverordnung finden Sie unter: www.bbraun.de/dsgvo
_______________________________________________________________
The information contained in this communication is confidential, may be 
attorney-client privileged, may constitute inside information, and is intended 
only for the use of the addressee. It is the property of the company of the 
sender of this e-mail. Unauthorized use, disclosure, or copying of this 
communication or any part thereof is strictly prohibited and may be unlawful. 
If you have received this communication in error, please notify us immediately 
by return e-mail and destroy this communication and all copies thereof, 
including all attachments.