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

Timeout issue in consumer on broker death #546

Closed
4578395263256 opened this issue Sep 29, 2015 · 17 comments
Closed

Timeout issue in consumer on broker death #546

4578395263256 opened this issue Sep 29, 2015 · 17 comments

Comments

@4578395263256
Copy link

We are currently running release 1.6.1. During testing, we killed a broker by ipmi_rebooting it (so that it wouldn't be able to close the connection). This caused consumers that were reading from partitions for which that was the leader to get stuck for 3-4 minutes, at the end of which there was this message repeated once per partition

read tcp :9092: i/o timeout

and after that things recovered.

Based on the fact that the read timeout is only 30 seconds, it seems like it shouldn't take 3-4 minutes for things to get unstuck. Here are our settings:

kafkaConfig.ClientID = config.ClientID
kafkaConfig.Consumer.Fetch.Default = 16 * 1024 * 1024
kafkaConfig.Consumer.Fetch.Min = 8 * 1024 * 1024
kafkaConfig.Consumer.Fetch.Max = 64 * 1024 * 1024
kafkaConfig.ChannelBufferSize = 500
kafkaConfig.Consumer.Return.Errors = true
kafkaConfig.Metadata.Retry.Max = 10
kafkaConfig.Metadata.Retry.Backoff = 5 * time.Second

Thanks for any insight you can provide!

@wvanbergen
Copy link
Contributor

Can you set sarama.Logger and capture the output? That should give you more insight in what is going on.

My best guess is this:

  • Broker goes away
  • Sarama tries to consume from broker, but fails after timeout.
  • Sarama asks a different broker what broker it should now consume from now.
  • The other broker looks this up in Zookeeper. Zookeeper has not yet been updated** so it holds the same broker as before.
  • Sarama tries to connect to the broker and fails again after a timeout.
    Etc.

So basically, Sarama will continue to retry using the same broker for a partition until the leader of that partition has been updated in Zookeeper. When a kafka broker doesn't shut down properly, it doesn't have a chance to hand off leadership to another one, and the handoff is determined by the connection timeout of the Kafka broker to Zookeeper. You can configure it in the Kafka broker configuration file.

@4578395263256
Copy link
Author

Note things recovered around 20:03:25. The following is the sarama logging output. This part of the log goes from 19:57:03 to 20:07:01.

[Sarama] 2015/09/28 19:59:43 consumer/broker/1 disconnecting due to error processing FetchRequest: read tcp [DEAD BROKER IP]:9092: i/o timeout
[Sarama] 2015/09/28 20:00:18 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:00:48 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:01:18 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:01:48 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:02:18 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:02:48 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:18 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:25 Closed connection to broker [DEAD BROKER].[domain]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/166 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/146 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/36 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/226 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/186 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/56 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/236 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/126 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/86 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/26 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/176 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/206 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/106 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/246 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/156 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/116 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/6 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/16 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/216 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/66 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/136 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/96 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/46 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/196 finding new broker
[Sarama] 2015/09/28 20:03:27 consumer/[TOPIC NAME]/76 finding new broker
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 client/metadata fetching metadata for [[TOPIC NAME]] from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:27 consumer/broker/5 added subscription to [TOPIC NAME]/186
[Sarama] 2015/09/28 20:03:27 consumer/broker/8 added subscription to [TOPIC NAME]/36
[Sarama] 2015/09/28 20:03:27 consumer/broker/8 added subscription to [TOPIC NAME]/126
[Sarama] 2015/09/28 20:03:27 consumer/broker/2 added subscription to [TOPIC NAME]/246
[Sarama] 2015/09/28 20:03:27 consumer/broker/9 added subscription to [TOPIC NAME]/226
[Sarama] 2015/09/28 20:03:27 consumer/broker/0 added subscription to [TOPIC NAME]/146
[Sarama] 2015/09/28 20:03:27 consumer/broker/0 added subscription to [TOPIC NAME]/236
[Sarama] 2015/09/28 20:03:27 consumer/broker/0 added subscription to [TOPIC NAME]/56
[Sarama] 2015/09/28 20:03:28 consumer/broker/7 added subscription to [TOPIC NAME]/26
[Sarama] 2015/09/28 20:03:28 consumer/broker/3 added subscription to [TOPIC NAME]/166
[Sarama] 2015/09/28 20:03:28 consumer/broker/3 added subscription to [TOPIC NAME]/76
[Sarama] 2015/09/28 20:03:28 consumer/broker/4 added subscription to [TOPIC NAME]/86
[Sarama] 2015/09/28 20:03:28 consumer/broker/4 added subscription to [TOPIC NAME]/176
[Sarama] 2015/09/28 20:03:28 consumer/broker/6 added subscription to [TOPIC NAME]/106
[Sarama] 2015/09/28 20:03:28 consumer/broker/6 added subscription to [TOPIC NAME]/16
[Sarama] 2015/09/28 20:03:28 consumer/broker/6 added subscription to [TOPIC NAME]/196
[Sarama] 2015/09/28 20:03:28 consumer/broker/5 added subscription to [TOPIC NAME]/6
[Sarama] 2015/09/28 20:03:28 consumer/broker/5 added subscription to [TOPIC NAME]/96
[Sarama] 2015/09/28 20:03:29 consumer/broker/9 added subscription to [TOPIC NAME]/46
[Sarama] 2015/09/28 20:03:29 consumer/broker/9 added subscription to [TOPIC NAME]/136
[Sarama] 2015/09/28 20:03:29 consumer/broker/2 added subscription to [TOPIC NAME]/156
[Sarama] 2015/09/28 20:03:29 consumer/broker/2 added subscription to [TOPIC NAME]/66
[Sarama] 2015/09/28 20:03:29 consumer/broker/8 added subscription to [TOPIC NAME]/216
[Sarama] 2015/09/28 20:03:29 consumer/broker/7 added subscription to [TOPIC NAME]/206
[Sarama] 2015/09/28 20:03:29 consumer/broker/7 added subscription to [TOPIC NAME]/116
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:03:30 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/28 20:04:00 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092

@eapache
Copy link
Contributor

eapache commented Sep 29, 2015

I assume you've left Consumer.Retry.Backoff at its default of 2 seconds?

I notice you have set Consumer.Return.Errors = true, what does your error handler do when it receives an error?

@4578395263256
Copy link
Author

Yes, it should still be 2 seconds. For handling Errors, we have a select in a loop that in the error case logs the error and does a few other things for certain errors (offset out of range, channel closed) that shouldn't apply here. In any case it should be fast.

@eapache
Copy link
Contributor

eapache commented Sep 29, 2015

[Sarama] 2015/09/28 19:59:43 consumer/broker/1 disconnecting due to error processing FetchRequest: read tcp [DEAD BROKER IP]:9092: i/o timeout

This is the broker-consumer disconnecting after the first i/o timeout. The consumer then calls Close() on that broker before doing anything else.

[Sarama] 2015/09/28 20:03:25 Closed connection to broker [DEAD BROKER].[domain]:9092

This is the broker connection finally being closed, three and a half minutes later. This unblocks the consumer, permitting it to check its metadata and reassign the partitions to other brokers.

So the question here is: why is closing the broker taking 3 1/2 minutes? The Close method does three things that are potentially blocking:

  • takes the lock for that broker (should be fast, nothing holds this lock for very long)
  • waits for any pending requests to receive a response or time out (could be slow)
  • calls Close on the underlying network connection (should be fast most of the time)

So potentially there were a lot of other pending requests that needed to time out? They wouldn't be coming from this consumer, but if another consumer or producer or application was sharing the same broker reference then it could be clogging the pipe.

@4578395263256
Copy link
Author

The other things going on should be

  • 25 consumers talking to the dead broker, which just are consume calls and the same situation as above. So these should have the 30s read timeout right?

These following ones are called on the broker directly. I'm not sure what timeouts they would have. Would they also just be 30s?

  • CommitOffset calls for the consumers, as we have custom logic for commits. However these shouldn't be running in some of our components, which still experienced the delays
  • FetchOffset and Fetch calls - but should only happen at startup
  • GetAvailableOffsets calls, in order to monitor the latest offsets to track lag better

There aren't any producers involved.

By blocking the pipe, do you mean hitting the Net.MaxOpenRequests limit?

@eapache
Copy link
Contributor

eapache commented Sep 29, 2015

The read/write/connect timeouts all default to 30 seconds, affect all request/response types, and are set in the Net configuration.

Are the 25 consumers and other calls all sharing a single Client?

By blocking the pipe, do you mean hitting the Net.MaxOpenRequests limit?

Basically. If that is set to 5, then up to 5 requests can be open-waiting-for-a-response. Thus when Close is called on the broker, it has to wait for all 5 pending requests which (if they time out) takes 30 seconds each, thus blocking for 2m30s.

@4578395263256
Copy link
Author

Yes, they all share the same client. Oh, I did not know that they were all serial. So the ones in the pipeline would mostly hit the connect timeout in that case right?

So that could explain things. Do you recommend dropping MaxOpenRequests to 1 or just using many clients or lowering the connect timeout? (We should be able to guarantee much lower connect latency)

Is this something that you might try to improve the situation of or should we just use one of the above workarounds?

@eapache
Copy link
Contributor

eapache commented Sep 29, 2015

Oh, I did not know that they were all serial.

Yes, within a single connection to a single broker, requests are strictly serial. You can have multiple connections open to the same broker, but you need multiple clients for that.

This actually has implications for any request that can be slow. For example, if you are consuming on a low-volume topic with a Consumer.MaxWaitTime of 5 seconds, then while the broker is waiting that 5 seconds, every other request on that connection will be blocked and doing nothing.

So the ones in the pipeline would mostly hit the connect timeout in that case right?

Exactly.

just using many clients

Because of the point above about MaxWaitTime and the strict serialization, we historically used many clients in certain cases. However, you should probably only really need one now; the various Offset calls are presumably low-volume and not strictly latency-sensitive, so it's only the many consumers that will cause problems when sharing.

Which brings me to: why do you have 25 consumers in the same program? Multiple topics/partitions can be handled within a single consumer, so unless you're actually consuming the same partition from 25 different offsets simultaneously I'm not sure I see the use case.

lowering the connect timeout? (We should be able to guarantee much lower connect latency)

Yes, we tend to run with this lowered a bit from the default. It helps of course but is not a final solution, just an optimization.

@4578395263256
Copy link
Author

Sorry, I mean that we have one consumer with 256 PartitionConsumers attached (of which 25 were talking to the dead broker). It also turns out we have a separate call per partition to fetch offsets for tracking (I did not think that really mattered at the time, oops), and this happens up to every 10 seconds.

@eapache
Copy link
Contributor

eapache commented Sep 30, 2015

Sorry, I mean that we have one consumer with 256 PartitionConsumers attached

The consumer does the right thing in this case, batching requests together, so it is not the problem.

It also turns out we have a separate call per partition to fetch offsets for tracking ... and this happens up to every 10 seconds.

256 calls every 10 seconds could definitely be causing the queueing. They're fast calls, so in normal operation they probably don't have any effect, but when each one takes 30 seconds to timeout that could cause this.

@4578395263256
Copy link
Author

Added some custom logging to Sarama and got this, which suggests that grabbing the lock takes a long time. Not sure why that would be though.

[Sarama] 2015/09/30 17:32:11 consumer/broker/7 disconnecting due to error processing FetchRequest: read tcp 10.32.87.43:9092: i/o timeout
[Sarama] 2015/09/30 17:32:11 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() grab lock
[Sarama] 2015/09/30 17:32:46 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:33:16 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:33:46 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:34:16 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:34:46 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:35:16 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:35:46 client/metadata fetching metadata for all topics from broker [OTHER BROKER]:9092
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() grabbed lock
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() close responses
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() closed responses
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() got done
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() close done
[Sarama] 2015/09/30 17:35:47 Closed connection to broker [DEAD BROKER]:9092
[Sarama] 2015/09/30 17:35:47 [CUSTOM Broker 7 [DEAD BROKER]:9092 0xc20b8ca2a0] b.Close() exit
[Sarama] 2015/09/30 17:35:49 consumer/[TOPIC]/142 finding new broker

@eapache
Copy link
Contributor

eapache commented Sep 30, 2015

Oh of course, because the broker holds the lock while it writes into the response channel. So if the MaxOpenRequests queue is already full, it will hold the lock until it can write to the queue, which will involve the response loop again.

If you're able to easily test this, I'd appreciate knowing if #548 makes it any better?

@4578395263256
Copy link
Author

After two trials with #548, it looks the connection gets closed almost immediately (within the same second) and time to recover is <2 minutes. So the patch is really helpful. Perhaps a 1-2 minute recovery time is to be expected, and it should be good enough for us (though I'll poke around a bit more to see what exactly is going on).

@4578395263256
Copy link
Author

Looking at a specific incident with the new code:

The controller logs first mention at 18:33:44: INFO controller.KafkaController: [Controller 6]: Broker failure callback for 0
then they are pretty busy until 18:34:15 looking like they are dealing with the broker failure, after which they are pretty quiet for several minutes. So probably all the brokers got the new partition assignments by 18:34:15?

Sarama detected the timeout at 18:34:06 and then quickly closed the connection to the dead broker. Later it had these (not the full Sarama logs)
[Sarama] 2015/09/30 18:34:08 consumer/[TOPIC NAME]/244 finding new broker
[Sarama] 2015/09/30 18:34:20 consumer/[TOPIC NAME]/244 finding new broker
[Sarama] 2015/09/30 18:34:32 consumer/[TOPIC NAME]/244 finding new broker
[Sarama] 2015/09/30 18:34:44 consumer/[TOPIC NAME]/244 finding new broker
[Sarama] 2015/09/30 18:34:45 consumer/broker/1 added subscription to [TOPIC NAME]/244

It looks like finding a new broker didn't work for a while, even after the controller gave out all the partition assignments. Though it's definitely possible that kafka brokers are returning stale data and the controller logs are misleading.

Anyways it would be nice to reduce recovery time further, but #548 is really helpful and should be good enough for our use case.

@eapache
Copy link
Contributor

eapache commented Oct 1, 2015

#548 is in master and will be in the next release.

When a consumer partition repeatedly tries to find a new broker, that means it can't fetch new metadata either because the RefreshMetadata or Leader calls in partitionConsumer.dispatch have failed.

If you want to dig further, I'd look at what data those methods are getting back from the cluster, and what's actually in zookeeper during this window.

@eapache eapache closed this as completed Oct 1, 2015
@4578395263256
Copy link
Author

Thanks for all of your help!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

3 participants