You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
Version Information
Version of Akka.NET? 1.5.33
Which Akka.NET Modules? Akka.Streams.Kafka
Describe the bug
We use the KafkaConsumer.CommittablePartitionedSource running in kubernetes with autoscaling enabled. When Pods are spun up/down, it is possible for the SubSourceLogic to get out of sync with the TopicPartition assignment on the consumer, which results in an ex being thrown and the consumer shutting down. The PartitionsAssignedHandler and PartitionsRevokedHandler handlers are a side effect of _consumer.Consume(), and doesn't appear handle timing properly.
To Reproduce
Steps to reproduce the behavior:
NOTE: I added additional logging to KafkaConsumerActor in order to better capture current start during rebalancing
Run kafka locally using docker
Create a topic with 10 partitions, in this case I'm using a members topic
Run a producer so that messages are generated with a key to produce to all partitions
Start 3 instances of a consumer:
Start Consumer 1, all 10 partitions are assigned
consumer 1 - 10 assignments: members [[0]], members [[1]], members [[2]], members [[3]], members [[4]], members [[5]], members [[6]], members [[7]], members [[8]], members [[9]]
Start Consumer 2, partitions are reassigned to the new consumer
consumer 1 - 5 assignments: members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]
consumer 2 - 5 assignments: members [[5]], members [[6]], members [[7]], members [[8]], members [[9]]
Start Consumer 3, partitions are reassigned to the new consumer
consumer 1 - 4 assignments: members [[0]], members [[1]], members [[2]], members [[3]]
consumer 2 - 3 assignments: members [[4]], members [[5]], members [[6]]
consumer 3 - 3 assignments: members [[7]], members [[8]], members [[9]]
Stop Consumer 1, check logs for consumers 2 and 3 to see if they stopped. It may take multiple restarts of consumer 1 to trigger the error.
Expected behavior
The KafkaConsumerActor properly update the _requests and _requestors on PartitionsRevokedHandler so that it isn't expected to consume messages from partitions that are not longer assigned.
Actual behavior
When a rebalance happens, partitions are revoked and then new partitions are assigned.
First, the next _consumer.Consume() completes with zero records to process and zero assignments since the partitions have been consumed.
This can be seen with these logs from Consumer 2:
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Messages requested from: [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$j#1512204890], for: members [[2]]
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Delayed poll when messages requested, periodic: False
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Poll requested, periodic: False
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Starting poll with rebalancing: False, 5 requests: members [[4]], members [[0]], members [[2]], members [[3]], members [[1]], 5 assignments: members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Processing 0 records, 0 assignments:
The PartitionsRevokedHandler is then called, where the IPartitionEventHandler notifies the SubSourceLogic that partitions have been revoked and _rebalanceInProgress is set to true.
[13:53:40 INF] [cdc639ef-c511-47e1-8567-44e19debc586] Partitions revoked: members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]
Based on timing, a Poll message can be sent to KafkaConsumerActor after the revoke and before new partitions are assigned. These logs show that when _consumer.Consume() is called, the assignments are empty, but there are still 3 active _requests from the SubSourceLogic.
[13:53:40 INF] [1ea20efb-b0ad-4df0-bbe1-38abb67f7265] Poll requested, periodic: True
[13:53:40 INF] [1ea20efb-b0ad-4df0-bbe1-38abb67f7265] Starting poll with rebalancing: True, 5 requests: members [[4]], members [[0]], members [[2]], members [[3]], members [[1]], 0 assignments:
When the _consumer.Consume() completes, it returns records from the newly assigned partitions.
[13:53:40 INF] [1ea20efb-b0ad-4df0-bbe1-38abb67f7265] Processing 5 records, 3 assignments: members [[4]], members [[5]], members [[6]]
Inside ProcessResult() there is a check to validate that the messages that were consumed are from the partitions that were requested. However, due to timing during a rebalance, these can be out of sync and a System.ArgumentException is thrown.
System.ArgumentException: Unexpected records polled. Expected: [members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]], result: [members [[4]], members [[5]]], consumer assignment: [members [[4]], members [[5]], members [[6]]]
at Akka.Streams.Kafka.Stages.Consumers.Actors.KafkaConsumerActor`2.ProcessResult(Guid pollCorrelationId, IImmutableSet`1 partitionsToFetch, List`1 rawResult) in /Users/jblackburn/repos/github/akka/Akka.Streams.Kafka/src/Akka.Streams.Kafka/Stages/Consumers/Actors/KafkaConsumerActor.cs:line 658 at Akka.Streams.Kafka.Stages.Consumers.Actors.KafkaConsumerActor`2.Poll(Guid pollCorrelationId) in /Users/jblackburn/repos/github/akka/Akka.Streams.Kafka/src/Akka.Streams.Kafka/Stages/Consumers/Actors/KafkaConsumerActor.cs:line 588
The exception is handled with ProcessExceptions() and the KafkaConsumerActor is stopped
[13:53:40 INF] [1ea20efb-b0ad-4df0-bbe1-38abb67f7265] Sending failure to [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$g#765079443], [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$a#1997365679], [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$h#1166729880], [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$j#1512204890], [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$k#1142637371], [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$i#1325051480]. Error: Unexpected records polled. Expected: [members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]], result: [members [[4]], members [[5]]], consumer assignment: [members [[4]], members [[5]], members [[6]]]
[13:53:40 ERR] [1ea20efb-b0ad-4df0-bbe1-38abb67f7265] Exception when polling from consumer, KafkaConsumerActor actor: Unexpected records polled. Expected: [members [[0]], members [[1]], members [[2]], members [[3]], members [[4]]], result: [members [[4]], members [[5]]], consumer assignment: [members [[4]], members [[5]], members [[6]]]
Screenshots
N/A
Environment
Dotnet 8 running on Mac, with kafka running in docker desktop
Additional context
Add any other context about the problem here.
The text was updated successfully, but these errors were encountered:
Version Information
Version of Akka.NET? 1.5.33
Which Akka.NET Modules? Akka.Streams.Kafka
Describe the bug
We use the
KafkaConsumer.CommittablePartitionedSource
running in kubernetes with autoscaling enabled. When Pods are spun up/down, it is possible for theSubSourceLogic
to get out of sync with theTopicPartition
assignment on the consumer, which results in an ex being thrown and the consumer shutting down. ThePartitionsAssignedHandler
andPartitionsRevokedHandler
handlers are a side effect of_consumer.Consume()
, and doesn't appear handle timing properly.To Reproduce
Steps to reproduce the behavior:
NOTE: I added additional logging to
KafkaConsumerActor
in order to better capture current start during rebalancingRun kafka locally using docker
Create a topic with 10 partitions, in this case I'm using a
members
topicRun a producer so that messages are generated with a key to produce to all partitions
Start 3 instances of a consumer:
Stop Consumer 1, check logs for consumers 2 and 3 to see if they stopped. It may take multiple restarts of consumer 1 to trigger the error.
Expected behavior
The
KafkaConsumerActor
properly update the_requests
and_requestors
onPartitionsRevokedHandler
so that it isn't expected to consume messages from partitions that are not longer assigned.Actual behavior
When a rebalance happens, partitions are revoked and then new partitions are assigned.
First, the next
_consumer.Consume()
completes with zero records to process and zero assignments since the partitions have been consumed.This can be seen with these logs from Consumer 2:
[13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Messages requested from: [akka://KafkaStream/user/members-consumer/StreamSupervisor-1/$$j#1512204890], for: members [[2]] [13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Delayed poll when messages requested, periodic: False [13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Poll requested, periodic: False [13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Starting poll with rebalancing: False, 5 requests: members [[4]], members [[0]], members [[2]], members [[3]], members [[1]], 5 assignments: members [[0]], members [[1]], members [[2]], members [[3]], members [[4]] [13:53:40 INF] [2cadbea5-c6a8-40cf-ad57-3a757c315b7b] Processing 0 records, 0 assignments:
The
PartitionsRevokedHandler
is then called, where theIPartitionEventHandler
notifies theSubSourceLogic
that partitions have been revoked and_rebalanceInProgress
is set totrue
.Based on timing, a
Poll
message can be sent toKafkaConsumerActor
after the revoke and before new partitions are assigned. These logs show that when_consumer.Consume()
is called, the assignments are empty, but there are still 3 active_requests
from theSubSourceLogic
.When the
_consumer.Consume()
completes, it returns records from the newly assigned partitions.Inside
ProcessResult()
there is a check to validate that the messages that were consumed are from the partitions that were requested. However, due to timing during a rebalance, these can be out of sync and aSystem.ArgumentException
is thrown.The exception is handled with
ProcessExceptions()
and theKafkaConsumerActor
is stoppedScreenshots
N/A
Environment
Dotnet 8 running on Mac, with kafka running in docker desktop
Additional context
Add any other context about the problem here.
The text was updated successfully, but these errors were encountered: