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

Randall Hauch commented on KAFKA-8370:
--------------------------------------

Verified that when setting {{config.storage.replication.factor=3}} in the 
Connect distributed worker configuration and starting Connect with only one 
Kafka broker, not pre-creating the internal config topic so that the worker 
tries to create the topic, the Connect worker fails with:

{code}
Replication factor: 3 larger than available brokers: 1
{code}

Here's the end of the worker log:

{noformat}
[2020-10-19 09:53:07,100] ERROR [Worker clientId=connect-1, 
groupId=connect-cluster] Uncaught exception in herder work thread, exiting:  
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:302)
org.apache.kafka.connect.errors.ConnectException: Error while attempting to 
create/find topic(s) 'connect-configs'
        at 
org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:321)
        at 
org.apache.kafka.connect.storage.KafkaConfigBackingStore$1.run(KafkaConfigBackingStore.java:501)
        at 
org.apache.kafka.connect.util.KafkaBasedLog.start(KafkaBasedLog.java:129)
        at 
org.apache.kafka.connect.storage.KafkaConfigBackingStore.start(KafkaConfigBackingStore.java:268)
        at 
org.apache.kafka.connect.runtime.AbstractHerder.startServices(AbstractHerder.java:130)
        at 
org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:288)
        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:1149)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.util.concurrent.ExecutionException: 
org.apache.kafka.common.errors.InvalidReplicationFactorException: Replication 
factor: 3 larger than available brokers: 1.
        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:260)
        at 
org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:282)
        ... 10 more
Caused by: org.apache.kafka.common.errors.InvalidReplicationFactorException: 
Replication factor: 3 larger than available brokers: 1.
[2020-10-19 09:53:07,109] INFO Stopped 
http_8083@3de10c4e{HTTP/1.1,[http/1.1]}{0.0.0.0:8083} 
(org.eclipse.jetty.server.AbstractConnector:380)
[2020-10-19 09:53:07,109] INFO node0 Stopped scavenging 
(org.eclipse.jetty.server.session:158)
[2020-10-19 09:53:07,612] INFO HV000001: Hibernate Validator 6.0.17.Final 
(org.hibernate.validator.internal.util.Version:21)
[2020-10-19 09:53:07,887] INFO Started 
o.e.j.s.ServletContextHandler@7fd9b3b0{/,null,AVAILABLE} 
(org.eclipse.jetty.server.handler.ContextHandler:825)
[2020-10-19 09:53:07,888] INFO REST resources initialized; server is started 
and ready to handle requests 
(org.apache.kafka.connect.runtime.rest.RestServer:324)
[2020-10-19 09:53:07,888] INFO Kafka Connect started 
(org.apache.kafka.connect.runtime.Connect:57)
[2020-10-19 09:53:07,888] INFO Kafka Connect stopping 
(org.apache.kafka.connect.runtime.Connect:67)
[2020-10-19 09:53:07,889] INFO Stopping REST server 
(org.apache.kafka.connect.runtime.rest.RestServer:332)
[2020-10-19 09:53:07,889] INFO REST server stopped 
(org.apache.kafka.connect.runtime.rest.RestServer:349)
[2020-10-19 09:53:07,889] INFO [Worker clientId=connect-1, 
groupId=connect-cluster] Herder stopping 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:676)
[2020-10-19 09:53:12,889] INFO [Worker clientId=connect-1, 
groupId=connect-cluster] Herder stopped 
(org.apache.kafka.connect.runtime.distributed.DistributedHerder:696)
[2020-10-19 09:53:12,890] INFO Kafka Connect stopped 
(org.apache.kafka.connect.runtime.Connect:72)
{noformat}

Also verified via manual testing that after manually creating the internal 
config topic for Connect on a broker with 1 node, then setting 
{{config.storage.replication.factor=3}} in the Connect distributed worker 
configuration and starting Connect results in the Connect worker starting 
successfully. This is because the AdminClient is throwing a 
TopicExistsException before it checks the replication factor for the internal 
topics, and this is the desired behavior.

> Kafka Connect should check for existence of internal topics before attempting 
> to create them
> --------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8370
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8370
>             Project: Kafka
>          Issue Type: Bug
>          Components: KafkaConnect
>    Affects Versions: 0.11.0.0
>            Reporter: Randall Hauch
>            Assignee: Randall Hauch
>            Priority: Major
>
> The Connect worker doesn't current check for the existence of the internal 
> topics, and instead is issuing a CreateTopic request and handling a 
> TopicExistsException. However, this can cause problems when the number of 
> brokers is fewer than the replication factor, *even if the topic already 
> exists* and the partitions of those topics all remain available on the 
> remaining brokers.
> One problem of the current approach is that the broker checks the requested 
> replication factor before checking for the existence of the topic, resulting 
> in unexpected exceptions when the topic does exist:
> {noformat}
> connect      | [2019-05-14 19:24:25,166] ERROR Uncaught exception in herder 
> work thread, exiting:  
> (org.apache.kafka.connect.runtime.distributed.DistributedHerder)
> connect      | org.apache.kafka.connect.errors.ConnectException: Error while 
> attempting to create/find topic(s) 'connect-offsets'
> connect      |        at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:255)
> connect      |        at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore$1.run(KafkaOffsetBackingStore.java:99)
> connect      |        at 
> org.apache.kafka.connect.util.KafkaBasedLog.start(KafkaBasedLog.java:127)
> connect      |        at 
> org.apache.kafka.connect.storage.KafkaOffsetBackingStore.start(KafkaOffsetBackingStore.java:109)
> connect      |        at 
> org.apache.kafka.connect.runtime.Worker.start(Worker.java:164)
> connect      |        at 
> org.apache.kafka.connect.runtime.AbstractHerder.startServices(AbstractHerder.java:114)
> connect      |        at 
> org.apache.kafka.connect.runtime.distributed.DistributedHerder.run(DistributedHerder.java:214)
> connect      |        at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> connect      |        at 
> java.util.concurrent.FutureTask.run(FutureTask.java:266)
> connect      |        at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> connect      |        at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> connect      |        at java.lang.Thread.run(Thread.java:748)
> connect      | Caused by: java.util.concurrent.ExecutionException: 
> org.apache.kafka.common.errors.InvalidReplicationFactorException: Replication 
> factor: 3 larger than available brokers: 2.
> connect      |        at 
> org.apache.kafka.common.internals.KafkaFutureImpl.wrapAndThrow(KafkaFutureImpl.java:45)
> connect      |        at 
> org.apache.kafka.common.internals.KafkaFutureImpl.access$000(KafkaFutureImpl.java:32)
> connect      |        at 
> org.apache.kafka.common.internals.KafkaFutureImpl$SingleWaiter.await(KafkaFutureImpl.java:89)
> connect      |        at 
> org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:260)
> connect      |        at 
> org.apache.kafka.connect.util.TopicAdmin.createTopics(TopicAdmin.java:228)
> connect      |        ... 11 more
> connect      | Caused by: 
> org.apache.kafka.common.errors.InvalidReplicationFactorException: Replication 
> factor: 3 larger than available brokers: 2.
> connect      | [2019-05-14 19:24:25,168] INFO Kafka Connect stopping 
> (org.apache.kafka.connect.runtime.Connect)
> {noformat}
> Instead of always issuing a CreateTopic request, the worker's admin client 
> should first check whether the topic exists, and if not *then* attempt to 
> create the topic.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to