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

ASF GitHub Bot commented on KAFKA-6250:
---------------------------------------

GitHub user gavrie opened a pull request:

    https://github.com/apache/kafka/pull/4247

    KAFKA-6250: Use existing internal topics without requiring ACL

    When using Kafka Connect with a cluster that doesn't allow the user to
    create topics (due to ACL configuration), Connect fails when trying to
    create its internal topics, even if these topics already exist. This is
    incorrect behavior according to the documentation, which mentions that
    R/W access should be enough.
    
    This happens specifically when using Aiven Kafka, which does not permit
    creation of topics via the Kafka Admin Client API.
    
    The patch ignores the returned error, similar to the behavior for older
    brokers that don't support the API.
    
    ### Committer Checklist (excluded from commit message)
    - [ ] Verify design and implementation 
    - [ ] Verify test coverage and CI build status
    - [ ] Verify documentation (including upgrade notes)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/gavrie/kafka trunk

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/4247.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4247
    
----
commit 0b17d56257784d9def1418ab87650cd240892227
Author: Gavrie Philipson <gav...@philipson.co.il>
Date:   2017-11-22T06:56:28Z

    KAFKA-6250: Use existing internal topics without requiring ACL
    
    When using Kafka Connect with a cluster that doesn't allow the user to
    create topics (due to ACL configuration), Connect fails when trying to
    create its internal topics, even if these topics already exist. This is
    incorrect behavior according to the documentation, which mentions that
    R/W access should be enough.
    
    This happens specifically when using Aiven Kafka, which does not permit
    creation of topics via the Kafka Admin Client API.
    
    The patch ignores the returned error, similar to the behavior for older
    brokers that don't support the API.

----


> Kafka Connect requires permission to create internal topics even if they exist
> ------------------------------------------------------------------------------
>
>                 Key: KAFKA-6250
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6250
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 0.11.0.1, 1.0.0
>            Reporter: Gavrie Philipson
>
> When using Kafka Connect with a cluster that doesn't allow the user to create 
> topics (due to ACL configuration), Connect fails when trying to create its 
> internal topics, even if these topics already exist.
> This happens specifically when using hosted [Aiven 
> Kafka|https://aiven.io/kafka], which does not permit creation of topics via 
> the Kafka Admin Client API.
> The problem is that Connect tries to create the topics, and ignores some 
> specific errors such as topics that already exist, but not authorization 
> errors.
> This is what happens:
> {noformat}
> 2017-11-21 15:57:24,176 [DistributedHerder] ERROR DistributedHerder:206 - 
> Uncaught exception in herder work thread, exiting:
> org.apache.kafka.connect.errors.ConnectException: Error while attempting to 
> create/find topic(s) 'connect-offsets'
>       at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:245)
>       at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore$1.run(KafkaOffsetBackingStore.java:99)
>       at 
> org.apache.kafka.connect.util.KafkaBasedLog.start(KafkaBasedLog.java:126)
>       at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore.start(KafkaOffsetBackingStore.java:109)
>       at org.apache.kafka.connect.runtime.Worker.start(Worker.java:146)
>       at 
> org.apache.kafka.connect.runtime.AbstractHerder.startServices(AbstractHerder.java:99)
>       at 
> org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:194)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.util.concurrent.ExecutionException: 
> org.apache.kafka.common.errors.ClusterAuthorizationException: Cluster 
> authorization failed.
>       at 
> org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
>       at 
> org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
>       at 
> org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
>       at 
> org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:213)
>       at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:226)
>       ... 11 more
> Caused by: org.apache.kafka.common.errors.ClusterAuthorizationException: 
> Cluster authorization failed.
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to