The log is showing that the Connect worker is trying to make sure it has read the entire log and gets to offset 119, but some other worker says it has read to offset 169. The two are in inconsistent states, so the one that seems to be behind will not start work with potentially outdated configuration info.
Since it is logging that it *finished* reading to the end of the log, the steps to check the offset and validate we've read those offsets have been met. That seems to indicate the two workers aren't looking at the same data. One case I'm aware of where this can happen is if you have two connect workers that are configured to be in the same Connect worker group (their worker configs have the same group.id) but they are actually looking at different config topics (config.storage.topic is different for the two workers). Sometimes people make this mistake when they start running multiple Connect clusters but forget to make some of the default settings unique. I'd probably start by looking into that possibility to debug this issue. -Ewen On Wed, Mar 21, 2018 at 10:15 PM, Ziliang Chen <zlchen....@gmail.com> wrote: > Hi, > > I have 2 Kafka Connect instances runs in 2 boxes which forms a Kafka > Connect Cluster. One of the instance seems doing the re-balance repeatedly > in a dead loop without running the actual Sink task, the other works fine. > The following is the output message in the console. > > May I ask if you have ever encountered similar issue before ? > > Thank you so much ! > > > [2018-03-21 14:53:15,671] WARN Catching up to assignment's config offset. > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:762) > > [2018-03-21 14:53:15,671] INFO Current config state offset 119 is behind > > group assignment 169, reading to end of config log > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:807) > > [2018-03-21 14:53:16,046] INFO Finished reading to end of log and updated > > config snapshot, new config log offset: 119 > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:811) > > [2018-03-21 14:53:16,046] INFO Current config state offset 119 does not > > match group assignment 169. Forcing rebalance. > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:786) > > [2018-03-21 14:53:16,046] INFO Rebalance started > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1214) > > [2018-03-21 14:53:16,046] INFO Wasn't unable to resume work after last > > rebalance, can skip stopping connectors and tasks > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1246) > > [2018-03-21 14:53:16,046] INFO [Worker clientId=connect-1, > > groupId=kafka-connect-NA-sink] (Re-)joining group > > (org.apache.kafka.clients.consumer.internals.AbstractCoordinator:336) > > [2018-03-21 14:53:16,172] INFO [Worker clientId=connect-1, > > groupId=kafka-connect-NA-sink] Successfully joined group with generation > 14 > > (org.apache.kafka.clients.consumer.internals.AbstractCoordinator:341) > > [2018-03-21 14:53:16,173] INFO Joined group and got assignment: > > Assignment{error=0, > > leader='connect-1-3b12c02b-070e-431b-9ec9-924cffd2b6dc', leaderUrl=' > > http://1.1.1.1:8083/', offset=169, connectorIds=[Send_To_NA], > > taskIds=[Send_To_NA-1, Send_To_NA-3, Send_To_NA-5, Send_To_NA-7, > > Send_To_NA-9, Send_To_NA-11, Send_To_NA-13, Send_To_NA-15, Send_To_NA-17, > > Send_To_NA-19, Send_To_NA-21]} > > (org.apache.kafka.connect.runtime.distributed.DistributedHerder:1192 > > > > -- > Regards, Zi-Liang > > Mail:zlchen....@gmail.com >