-
Notifications
You must be signed in to change notification settings - Fork 1.8k
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
Consumer facing higher latency after a preferred leader election #1927
Comments
@ppatierno a few questions:
I wonder if since KIP-392 and the ability of fetching from follower, perhaps we might have become sticky on the replica we're consuming on and perhaps we are not correctly handling the (default) LeaderSelector responses to change which broker we're consuming from |
@dnwe thanks for jumping into it so quickly! |
@dnwe FYI I was still using Sarama 1.27.2 before but even switching to the new 1.28.0 (that implements KIP-392: Allow consumers to fetch from closest replica) I see the same problem. |
@dnwe I dig into it more and it's now clear that the Sarama client is not connecting to broker 0 anymore even when it comes again and is the leader for partition 0. Initially everything is fine ... 2021/04/28 17:18:17 The canary topic __strimzi_canary already exists
2021/04/28 17:18:17 Metadata for __strimzi_canary topic
2021/04/28 17:18:17 {ID:0 Leader:0 Replicas:[0 1 2] Isr:[0 1 2] OfflineReplicas:[]}
2021/04/28 17:18:17 {ID:2 Leader:2 Replicas:[2 0 1] Isr:[2 0 1] OfflineReplicas:[]}
2021/04/28 17:18:17 {ID:1 Leader:1 Replicas:[1 2 0] Isr:[1 2 0] OfflineReplicas:[]}
2021/04/28 17:18:17 Sending message: value={"producerId":"strimzi-canary-client","messageId":4,"timestamp":1619623097756} on partition=0
[Sarama] 2021/04/28 17:18:17 ***** FetchRequest for broker 2
[Sarama] 2021/04/28 17:18:17 ***** partition 2
[Sarama] 2021/04/28 17:18:17 ***** partition 2
[Sarama] 2021/04/28 17:18:18 ***** FetchRequest for broker 1
[Sarama] 2021/04/28 17:18:18 ***** partition 1
[Sarama] 2021/04/28 17:18:18 ***** partition 1
[Sarama] 2021/04/28 17:18:18 ***** FetchRequest for broker 0
[Sarama] 2021/04/28 17:18:18 ***** partition 0
[Sarama] 2021/04/28 17:18:18 ***** partition 0 then when broker 0 goes down ... broker handler 1 is fetching partition 0 and 1 (broker 1 is leader for both now) 2021/04/28 17:19:27 {ID:0 Leader:1 Replicas:[0 1 2] Isr:[1 2 0] OfflineReplicas:[]}
2021/04/28 17:19:27 {ID:2 Leader:2 Replicas:[2 0 1] Isr:[2 1 0] OfflineReplicas:[]}
2021/04/28 17:19:27 {ID:1 Leader:1 Replicas:[1 2 0] Isr:[1 2 0] OfflineReplicas:[]}
2021/04/28 17:19:27 Sending message: value={"producerId":"strimzi-canary-client","messageId":43,"timestamp":1619623167869} on partition=0
[Sarama] 2021/04/28 17:19:27 ***** FetchRequest for broker 2
[Sarama] 2021/04/28 17:19:27 ***** partition 2
[Sarama] 2021/04/28 17:19:27 ***** partition 2
[Sarama] 2021/04/28 17:19:28 ***** FetchRequest for broker 1
[Sarama] 2021/04/28 17:19:28 ***** partition 1
[Sarama] 2021/04/28 17:19:28 ***** partition 0
[Sarama] 2021/04/28 17:19:28 ***** partition 0
[Sarama] 2021/04/28 17:19:28 ***** partition 1 finally broker 0 comes back again and it becomes leader but broker handler 1 is fetching partition 0 and 1 so it's actually getting messages for partition 0 from a follower replica 2021/04/28 17:21:12 The canary topic __strimzi_canary already exists
2021/04/28 17:21:12 Metadata for __strimzi_canary topic
2021/04/28 17:21:12 {ID:0 Leader:0 Replicas:[0 1 2] Isr:[1 2 0] OfflineReplicas:[]}
2021/04/28 17:21:12 {ID:2 Leader:2 Replicas:[2 0 1] Isr:[2 1 0] OfflineReplicas:[]}
2021/04/28 17:21:12 {ID:1 Leader:1 Replicas:[1 2 0] Isr:[1 2 0] OfflineReplicas:[]}
2021/04/28 17:21:12 Sending message: value={"producerId":"strimzi-canary-client","messageId":106,"timestamp":1619623272755} on partition=0
2021/04/28 17:21:12 Message sent: partition=0, offset=35, duration=9 ms
2021/04/28 17:21:12 Sending message: value={"producerId":"strimzi-canary-client","messageId":107,"timestamp":1619623272764} on partition=1
[Sarama] 2021/04/28 17:21:12 ***** FetchRequest for broker 2
[Sarama] 2021/04/28 17:21:12 ***** partition 2
[Sarama] 2021/04/28 17:21:12 ***** partition 2
[Sarama] 2021/04/28 17:21:13 ***** FetchRequest for broker 1
[Sarama] 2021/04/28 17:21:13 ***** partition 1
[Sarama] 2021/04/28 17:21:13 ***** partition 0
[Sarama] 2021/04/28 17:21:13 ***** partition 1
[Sarama] 2021/04/28 17:21:13 ***** partition 0 Referring KIP-392 and checked that:
Do we should expect the broker returning an error when the client tries to fetch from broker 1 that is not leader for partition 0 anymore? |
Going further we saw that on the broker side there is this condition:
So the logic is about fetching from the leader if it’s a follower (so another broker) or a consumer but that And as you can see for version >= 11 so Kafka 2.6 and above, it’s never empty but returned The broker allows continuing to fetch from a follower even if the leader came back again. |
We definitely do periodically refresh metadata, but I imagine we don't stop and change a partition consumer if it's working. Thanks for all the good debug! |
@dnwe yes you are right (of course! ;-)) even after a metadata refresh nothing is changed. |
@dnwe as follow up to this, taking a look at the Sarma source code, I see there is a kind of mechanism to exit the session when the number of topic partitions change on metedata refresh (see |
Yeah...so that was added under #1525 and then corrected under #1578 though it probably still needs revisiting as the "pause" ticker matching the refresh frequency isn't really ideal, this would ideally be event-driven from MetadataResponses arriving. So yes, you could add some code there to look at the client.Leader(...) for each topicpartition and take action if that has changed between ticks, as you mention that would match what we're currently doing for changes in topicpartitions numbers. However, I wonder if a simpler solution would be to modify the Sarama KIP-392 code slightly to handle this. As we already have a i.e., diff --git a/consumer.go b/consumer.go
index 8681f35..46c2613 100644
--- a/consumer.go
+++ b/consumer.go
@@ -838,10 +838,12 @@ func (bc *brokerConsumer) handleResponses() {
child.responseResult = nil
if result == nil {
- if child.preferredReadReplica >= 0 && bc.broker.ID() != child.preferredReadReplica {
- // not an error but needs redispatching to consume from prefered replica
- child.trigger <- none{}
- delete(bc.subscriptions, child)
+ if preferredBroker, err := child.preferredBroker(); err == nil {
+ if bc.broker.ID() != preferredBroker.ID() {
+ // not an error but needs redispatching to consume from prefered replica
+ child.trigger <- none{}
+ delete(bc.subscriptions, child)
+ }
}
continue
} I'll push this up to a branch and perhaps you could give it a spin? |
Historically (before protocol version 11) if we attempted to consume from a follower, we would get a NotLeaderForPartition response and move our consumer to the new leader. However, since v11 the Kafka broker treats us just like any other follower and permits us to consume from any replica and it is up to us to monitor metadata to determine when the leadership has changed. Modifying the handleResponse func to check the topic partition leadership against the current broker (in the absence of a preferredReadReplica) and trigger a re-create of the consumer for that partition Contributes-to: #1927
@ppatierno please can you let me know if |
@dnwe that was a simpler solution, even because the -1 returned in the preferred_read_replica should mean "use the leader" which is exactly what the |
Excellent, thanks for confirming. I think we're due a new release imminently due to some other important bugfixes and new KIPs. I'd like to add a unittest to cover this usecase too, but I imagine we can tag a new release within the next few days (cc @bai) |
Historically (before protocol version 11) if we attempted to consume from a follower, we would get a NotLeaderForPartition response and move our consumer to the new leader. However, since v11 the Kafka broker treats us just like any other follower and permits us to consume from any replica and it is up to us to monitor metadata to determine when the leadership has changed. Modifying the handleResponse func to check the topic partition leadership against the current broker (in the absence of a preferredReadReplica) and trigger a re-create of the consumer for that partition Contributes-to: #1927
TestConsumeMessagesTrackLeader ensures that in the event that leadership of a topicPartition changes and no preferredReadReplica is specified, the consumer connects back to the new leader to resume consumption and doesn't continue consuming from the follower. See #1927
TestConsumeMessagesTrackLeader ensures that in the event that leadership of a topicPartition changes and no preferredReadReplica is specified, the consumer connects back to the new leader to resume consumption and doesn't continue consuming from the follower. See #1927
@ppatierno thanks for the bug report — should be fixed in v1.29.0 |
@dnwe In my case, I had rack awareness on for testing purposes only. When I disabled this in the consumer group, the issue went away. |
also seeing issues after bisecting to 1aac8e5 -- throughput drops to approximately 0 on consumers who are rack-aware, but not on consumers with rack awareness disabled. |
😱 please can you raise an issue, we should fix asap |
Opening issue. |
Hi,
as part of the Strimzi project, we have a canary application sending and receiving messages to test that the Kafka cluster is working fine. This canary tool is using Sarama and we are facing the following "problem".
Imagine a cluster with 3 brokers, and the canary application creates a topic with 3 partition, one on each broker.
A normal canary flow shows the following recurring log:
It shows a latency that quite often is around 200 ms.
Restarting the broker 0 produces the following effect:
The replica 0 is offline (because broker 0 is down) but canary keeps going to send messages to partition 0 that now has the leader on broker 1. The latency is still ok.
When broker 0 restarts, it doesn’t become the leader for partition 0 immediately and meanwhile the canary still sends to partition 0 (leader on broker 1) and latency is still ok.
When finally the preferred leader election happens and broker 0 is again the leader for partition 0, the canary is affected by higher latency on partition 0. You can see from around 200 ms to 940 ms and this latency has an average of 700/800 ms with canary keep going to send/receive messages.
Restarting the canary helps.
Tracing down some Sarama stuff, we noticed that when broker 0 goes off, something like this happens:
so it's clear to me that now the consumer is consuming partition 0 from broker 1 (that is actually the leader).
But when the broker 0 comes back again and the new leader is elected, we see:
[Sarama] 2021/04/28 11:48:58 producer/broker/0 state change to [closing] because dial tcp 127.0.0.1:9092: connect: connection refused [Sarama] 2021/04/28 11:48:58 producer/leader/__strimzi_canary/0 state change to [retrying-1] [Sarama] 2021/04/28 11:48:58 producer/leader/__strimzi_canary/0 abandoning broker 0 [Sarama] 2021/04/28 11:48:58 producer/broker/0 input chan closed [Sarama] 2021/04/28 11:48:58 producer/broker/0 shut down [Sarama] 2021/04/28 11:48:58 client/metadata fetching metadata for [__strimzi_canary] from broker my-cluster-kafka-brokers.default.svc:9094 [Sarama] 2021/04/28 11:48:59 producer/leader/__strimzi_canary/0 selected broker 1 [Sarama] 2021/04/28 11:48:59 producer/broker/1 state change to [open] on __strimzi_canary/0 [Sarama] 2021/04/28 11:48:59 producer/leader/__strimzi_canary/0 state change to [flushing-1] [Sarama] 2021/04/28 11:48:59 producer/leader/__strimzi_canary/0 state change to [normal]
So only the producer got that leader is changed and moves to send to broker 0 but there is no log from the consumer perspective to get that now the leader is 0 and should read from there.
Other than missing this log, I don't understand how the consumer is then consuming and why the higher latency from now on.
The text was updated successfully, but these errors were encountered: