In case anyone else runs into this issue: Turning on TRACE level logs revealed that the config topic we were using was auto-created to have 12 partitions. As stated in the Kafka Connect User Guide (http://docs.confluent.io/3.1.2/connect/userguide.html), the internal topic used to store configs (config.storage.topic) must only be a single partition. This explains why the consumers could never catch up / reach the end of the config topic. Simply switching config.storage.topic to point to a new topic with a single partition resolved this issue.
On Wed, Jan 4, 2017 at 6:18 PM, Ewen Cheslack-Postava <e...@confluent.io> wrote: > Aside from the logs you already have, the best suggestion I have is to > enable trace level logging and try to reproduce -- there are some trace > level logs in the KafkaBasedLog class that this uses which might reveal > something. But it could be an issue in the consumer as well -- it sounds > like it is getting hung up and not actually getting in sync to the end of > the log, but obviously that shouldn't happen since the consumer should be > continuously trying to read any new messages on the topic. > > -Ewen > > On Wed, Jan 4, 2017 at 9:32 AM, Willy Hoang < > willy.ho...@blueapron.com.invalid> wrote: > > > I'm also running into this issue whenever I try to scale up from 1 worker > > to multiple. I found that I can sometimes hack around this by > > (1) waiting for the second worker to come up and start spewing out these > > log messages and then > > (2) sending a request to the REST API to update one of my connectors. > > > > I'm assuming that the update appends a message to the config topic which > > then triggers all the workers to catch up to the newest offset. > > > > "The fact that your worker isn't able to catch up probably indicates > > a connectivity issue or possibly even some misconfiguration" > > > > Since I'm able to sometimes trick the worker into catching up and I see > > multiple workers working smoothly, I'm pretty confident that my > > configurations are all correct. Any advice on how to debug what the > > underlying issue could be? > > > > P.S. The version of Kafka Connect I'm running is > > {"version":"0.10.0.0-cp1","commit":"7aeb2e89dbc741f6"} > > On Sat, Dec 17, 2016 at 7:55 PM, Ewen Cheslack-Postava < > e...@confluent.io> > > wrote: > > > > > The message > > > > > > > Wasn't unable to resume work after last rebalance > > > > > > means that you previous iterations of the rebalance were somehow > > behind/out > > > of sync with other members of the group, i.e. they had not read up to > the > > > same point in the config topic so it wouldn't be safe for this worker > (or > > > possibly the entire cluster if this worker was the leader) to resume > > work. > > > (I think there's a typo in the log message, it should say "wasn't > *able* > > to > > > resume work".) > > > > > > This message indicates the problem: > > > > > > > Catching up to assignment's config offset. > > > > > > The leader was using configs that were newer than this member, so it's > > not > > > safe for it to start its assigned work since it might be using outdated > > > configuration. When it tries to catch up, it continues trying to read > up > > > until the end of the config topic, which should be at least as far as > the > > > leader indicated its position was. (Another gap in logging: that > message > > > should really include the offset it is trying to catch up to, although > > you > > > can also check that manually since it'll always be trying to read to > the > > > end of the topic.) > > > > > > This catch up has a timeout which defaults to 3s (which is pretty > > > substantial given the rate at which configs tend to be written and > their > > > size). The fact that your worker isn't able to catch up probably > > indicates > > > a connectivity issue or possibly even some misconfiguration where one > > > worker is looking at one cluster/config topic, and the other is in the > > same > > > group in the same cluster but looking at a different cluster/config > topic > > > when reading configs. > > > > > > -Ewen > > > > > > On Fri, Dec 16, 2016 at 3:16 AM, Frank Lyaruu <flya...@gmail.com> > wrote: > > > > > > > Hi people, > > > > > > > > I've just deployed my Kafka Streams / Connect (I only use a connect > > sink > > > to > > > > mongodb) application on a cluster of four instances (4 containers on > 2 > > > > machines) and now it seems to get into a sort of rebalancing loop, > and > > I > > > > don't get much in mongodb, I've got a little bit of data at the > > > beginning, > > > > but no new data appears. > > > > > > > > The rest of the streams application seems to behave. > > > > > > > > This is what I get in my log, but at a pretty high speed (about 100 > per > > > > second): > > > > > > > > Current config state offset 3 is behind group assignment 5, reading > to > > > end > > > > of config log > > > > Joined group and got assignment: Assignment{error=0, > > > > leader='connect-2-8fb3bfc4-93f2-4d08-82df-8e7c4b99ec13', > leaderUrl='', > > > > offset=5, connectorIds=[KNVB-production-generation-99-person- > > mongosink], > > > > taskIds=[]} > > > > Successfully joined group NHV-production-generation-99- > > person-mongosink > > > > with generation 6 > > > > Successfully joined group KNVB-production-generation-99- > > person-mongosink > > > > with generation 6 > > > > Wasn't unable to resume work after last rebalance, can skip stopping > > > > connectors and tasks > > > > Rebalance started > > > > Wasn't unable to resume work after last rebalance, can skip stopping > > > > connectors and tasks > > > > (Re-)joining group KNVB-production-generation-99-person-mongosink > > > > Current config state offset 3 does not match group assignment 5. > > Forcing > > > > rebalance. > > > > Finished reading to end of log and updated config snapshot, new > config > > > log > > > > offset: 3 > > > > Finished reading to end of log and updated config snapshot, new > config > > > log > > > > offset: 3 > > > > Current config state offset 3 does not match group assignment 5. > > Forcing > > > > rebalance. > > > > Joined group and got assignment: Assignment{error=0, > > > > leader='connect-1-1893fd59-3ce8-4061-8131-ae36e58f5524', > leaderUrl='', > > > > offset=5, connectorIds=[], taskIds=[]} > > > > Current config state offset 3 is behind group assignment 5, reading > to > > > end > > > > of config log > > > > Successfully joined group KNVB-production-generation-99- > > person-mongosink > > > > with generation 6 > > > > (Re-)joining group KNVB-production-generation-99-person-mongosink > > > > Current config state offset 3 does not match group assignment 5. > > Forcing > > > > rebalance.Rebalance started > > > > Current config state offset 3 is behind group assignment 5, reading > to > > > end > > > > of config log > > > > Catching up to assignment's config offset. > > > > Successfully joined group NHV-production-generation-99- > > person-mongosink > > > > with generation 6 > > > > Joined group and got assignment: Assignment{error=0, > > > > leader='connect-2-8fb3bfc4-93f2-4d08-82df-8e7c4b99ec13', > leaderUrl='', > > > > offset=5, connectorIds=[], taskIds=[]} > > > > Catching up to assignment's config offset. > > > > Joined group and got assignment: Assignment{error=0, > > > > leader='connect-2-8fb3bfc4-93f2-4d08-82df-8e7c4b99ec13', > leaderUrl='', > > > > offset=5, connectorIds=[], taskIds=[]} > > > > (Re-)joining group NHV-production-generation-99-person-mongosink > > > > Wasn't unable to resume work after last rebalance, can skip stopping > > > > connectors and tasks > > > > Successfully joined group NHV-production-generation-99- > > person-mongosink > > > > with generation 6 > > > > Current config state offset 3 does not match group assignment 5. > > Forcing > > > > rebalance. > > > > Finished reading to end of log and updated config snapshot, new > config > > > log > > > > offset: 3 > > > > Current config state offset 3 does not match group assignment 5. > > Forcing > > > > rebalance. > > > > Rebalance started > > > > > > > > ... and so on.. > > > > > > > > Any ideas? > > > > > > > > regards, Frank > > > > > > > > > >