Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Endless loop while initializing a consumer group #1381

Closed
NeoHuang opened this issue May 28, 2019 · 13 comments
Closed

Endless loop while initializing a consumer group #1381

NeoHuang opened this issue May 28, 2019 · 13 comments
Labels
help wanted stale Issues and pull requests without any recent activity

Comments

@NeoHuang
Copy link

NeoHuang commented May 28, 2019

Versions

Sarama Version: 1.22.1 (ea9ab1c)
Kafka Version: 2.1.1
Go Version: go1.11.2

Configuration

What configuration values are you using for Sarama and Kafka?
for Sarama I simply set this config

	version, err := sarama.ParseKafkaVersion("1.22.1")
	if err != nil {
		panic(err)
	}

	config := sarama.NewConfig()
	config.Version = version
	config.Consumer.Return.Errors = true
Logs
[sarama] 2019/05/28 16:06:13 client/metadata fetching metadata for [testTopic] from broker localhost:9092
[sarama] 2019/05/28 16:06:13 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:13 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:13 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:13 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:13 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:14 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:14 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:14 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:14 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:14 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:14 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:14 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:14 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:14 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:16 client/coordinator retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:16 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:16 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:16 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:16 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:16 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:17 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:17 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:17 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:17 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:17 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:17 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:17 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:17 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:17 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:19 client/coordinator retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:19 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:19 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:19 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:19 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:19 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:20 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:20 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:20 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:20 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:20 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:20 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:20 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:20 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:20 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:22 client/coordinator retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:22 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:22 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:22 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:22 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:22 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:23 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:23 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:23 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:23 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:23 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:23 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:23 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:23 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:23 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:27 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:27 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:27 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:27 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:27 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:27 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:27 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:27 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:28 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:28 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:28 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:28 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:28 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:28 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:30 client/coordinator retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:30 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:30 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:30 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:30 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:30 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:30 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:30 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:30 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:31 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:31 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:31 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:31 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:31 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:31 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:33 client/coordinator retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:33 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:33 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:33 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:33 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:33 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:33 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:33 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:33 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:34 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:43 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:43 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:43 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:43 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:43 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
[sarama] 2019/05/28 16:06:45 client/coordinator retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:46 client/coordinator requesting coordinator for consumergroup kafka_test_DimapD from localhost:9092
[sarama] 2019/05/28 16:06:46 client/coordinator coordinator for consumer group kafka_test_DimapD is not available
[sarama] 2019/05/28 16:06:46 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:46 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:46 client/metadata retrying after 250ms... (3 attempts remaining)
[sarama] 2019/05/28 16:06:46 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:46 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:46 client/metadata retrying after 250ms... (2 attempts remaining)
[sarama] 2019/05/28 16:06:46 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:46 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:46 client/metadata retrying after 250ms... (1 attempts remaining)
[sarama] 2019/05/28 16:06:46 client/metadata fetching metadata for [__consumer_offsets] from broker localhost:9092
[sarama] 2019/05/28 16:06:46 client/metadata found some partitions to be leaderless
[sarama] 2019/05/28 16:06:46 client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...
Problem Description

I am trying to initialize a consumer group with simple setup from example code. After calling ConsumerGroup.Consume, I notice the Setup function seems not triggered at all. I traced down the issue and found out that the code stuck at retryNewSession func and more precisely https://github.com/Shopify/sarama/blob/ea9ab1c316850bee881a07bb2555ee8a685cd4b6/consumer_group.go#L189

from the log, looks like my test kafka cluster has some corrupt __consumer_offsets topic
from the code, looks like if RefreshCoordinator always returns error, then it will keep retrying regardless the retry settings. the call stack of retryNewSession will get deeper and deeper.

more importantly, I don't seem to find a way to catch this error. so the caller of Consumer will never notice this error which makes monitoring on such error impossible.

could you check if it's expected behavior? Thank you

@NeoHuang NeoHuang changed the title Endless loop while initialize a consumer group Endless loop while initializing a consumer group May 28, 2019
@NeoHuang
Copy link
Author

Hi guys, any update about this issue?

@FrancoisPoinsot
Copy link
Contributor

FrancoisPoinsot commented Aug 18, 2019

I see you are connecting to "localhost".
Is you kafka cluster a single node cluster running on your local machine?
Or is it some port-forwarding to a distant cluster?

If it is port-forwarding, be careful, it can mess up the way the kafka protocol works.
All you brokers have an "advertised host" that will be transmitted to clients that need to communicate with these brokers.
This "advertised host" need to be a functional host for you client.
You may have only forwarded "localhost". And if you are running an actual cluster of brokers, I am fairly sure that your brokers are not setup with advertised.host.name=localhost

But if it is not port forwarding, I guess what we have here is a simple kafka cluster for test purpose ?
Then you might as well restart the cluster from scratch. (clean up the data)
If you had any corruption on your __consumer_offsets topic, they will be solved this way.
This kind of corruption should not happen as soon as you will be working with a production-ready setup for you kafka cluster.

edit: But if i am wrong, please post the broker config.

@NeoHuang
Copy link
Author

NeoHuang commented Aug 18, 2019 via email

@d1egoaz
Copy link
Contributor

d1egoaz commented Aug 22, 2019

This PR introduced that code path, #1231 it also mentions problem related to __consumer_offsets.

Sadly, there are no unit tests for this.
It might or not be an issue regarding not decreasing the retries...

@d1egoaz
Copy link
Contributor

d1egoaz commented Aug 22, 2019

I could see the problem here:

https://github.com/Shopify/sarama/blob/99d5206b02d426e770d8cc65f578c3c1e19ab892/consumer_group.go#L181-L196

if there is an error in 1), 2) never decreases the retries, hence the endless loop

@d1egoaz
Copy link
Contributor

d1egoaz commented Aug 22, 2019

on top of that, it doesn't matter if the retries are not decreased, as it seems retryNewSession doesn't care about the retries, retries check are only performed in newSession, that will be only called if err := c.client.RefreshCoordinator(c.groupID) doesn't return an error

@d1egoaz
Copy link
Contributor

d1egoaz commented Aug 22, 2019

let's ping the original PR author to see if it has some idea about what's going on.
Help @thomaslee 🙏

@thomaslee
Copy link
Contributor

@d1egoaz good eye! I think you're right, this looks like a bug. Agree the line you pointed out is likely to be problematic, and at a glance it does seem like we should be making the recursive call with retries-1. As you say we probably also need handle the retries == 0 case somewhere in there too, and return the original error similar to what's going on here

Unfortunately it could be a week or two before I can have a shot at this, but on the surface it seems like an easy-ish change if somebody else wants to try before then.

Thanks again for tracking this down (and sorry for the trouble!)

@NOMORECOFFEE
Copy link

NOMORECOFFEE commented Nov 5, 2019

retryNewSession doesn't use context in select. If Retry.Backoff or retries is too big then this may be a problem.

 	select { 
 	case <-c.closed: 
 		return nil, ErrClosedConsumerGroup 
 	case <-time.After(c.config.Consumer.Group.Rebalance.Retry.Backoff): 
 	} 

@ghost
Copy link

ghost commented Feb 21, 2020

Thank you for taking the time to raise this issue. However, it has not had any activity on it in the past 90 days and will be closed in 30 days if no updates occur.
Please check if the master branch has already resolved the issue since it was raised. If you believe the issue is still valid and you would like input from the maintainers then please comment to ask for it to be reviewed.

@ghost ghost added the stale Issues and pull requests without any recent activity label Feb 21, 2020
@NeoHuang
Copy link
Author

Hi, I believe the bug is still there in master https://github.com/Shopify/sarama/blob/4ee86d9c4d49e2d434afa98c39f7db9276ff7d17/consumer_group.go#L198

here the retries is not decreased.

@ghost ghost removed the stale Issues and pull requests without any recent activity label Feb 21, 2020
@ghost
Copy link

ghost commented May 21, 2020

Thank you for taking the time to raise this issue. However, it has not had any activity on it in the past 90 days and will be closed in 30 days if no updates occur.
Please check if the master branch has already resolved the issue since it was raised. If you believe the issue is still valid and you would like input from the maintainers then please comment to ask for it to be reviewed.

@ghost ghost added the stale Issues and pull requests without any recent activity label May 21, 2020
@ghost ghost closed this as completed Mar 17, 2021
@peick
Copy link

peick commented Oct 25, 2021

I have the same issue. The consumer group client stays in the connection loop when there are no sufficient grants.

This issue was closed.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
help wanted stale Issues and pull requests without any recent activity
Projects
None yet
Development

No branches or pull requests

6 participants