Hi Florian,

Switching from a Zookeeper based cluster to a KRaft based one is not
currently supported. AFAIK that functionality should be coming in Kafka
3.4 (or possibly later).

Cheers,

Tom

On 16/05/2022 12:42, Florian Blumenstein wrote:
> 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.

Reply via email to