[
https://issues.apache.org/jira/browse/KAFKA-6306?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16278029#comment-16278029
]
zhaojianbo commented on KAFKA-6306:
-----------------------------------
It seem to be a bug. Look at the case:
1. The consumer start to rebalance.
2. The joinGroup phase is ok. (setting AbstractCoordinator.this.rejoinNeeded =
false; in method: JoinGroupResponseHandler.handle(....))
3. But the SyncGroup failed for some reasion such as the broker‘s network
inaccessible . (setting AbstractCoordinator.state = MemberState.UNJOINED; in
callback: RequestFutureListener.onFailure(…))
In this case, the rejoinNeeded = false and AbstractCoordinator.state =
MemberState.UNJOINED, the consumer will never rebalance again.
We have simulated the case, and reproduce the bug in the following:
1.Start two consumer and wait for the starting of consuming.
2.Drop the packet of the Kafka daemon binding port by applying the iptable’s
rules in order to trigger rebalancing
3.Clear the iptable’s rules and make the two consumer start to rebalance. But
Drop the packet of the Kafka daemon binding port by applying the iptable’s
rules again when JoinGroup phase is ok(which make
AbstractCoordinator.this.rejoinNeeded = false;), But still not start SyncGroup
phase. (To make the case above)
4.Wait for rebalance's timeout to make the SyncGroup phase failed which make
the AbstractCoordinator.state = MemberState.UNJOINED
5.Clear the iptable’s rules again, but the two consumer never recovery again.
Start to log the following:
_[2017-12-05 13:14:14,141] [main] WARN Auto-commit of offsets
{test_leader-4=OffsetAndMetadata{offset=35646295, metadata=''},
test_leader-2=OffsetAndMetadata{offset=35598088, metadata=''},
test_leader-3=OffsetAndMetadata{offset=35680898, metadata=''},
test_leader-0=OffsetAndMetadata{offset=35660648, metadata=''},
test_leader-1=OffsetAndMetadata{offset=35658968, metadata=''}} failed for group
test-group-123: Commit cannot be completed since the group has already
rebalanced and assigned the partitions to another member. This means that the
time between subsequent calls to poll() was longer than the configured
max.poll.interval.ms, which typically implies that the poll loop is spending
too much time message processing. You can address this either by increasing the
session timeout or by reducing the maximum size of batches returned in poll()
with max.poll.records.
(org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)_
> Auto-commit of offsets fail, and not recover forever...
> -------------------------------------------------------
>
> Key: KAFKA-6306
> URL: https://issues.apache.org/jira/browse/KAFKA-6306
> Project: Kafka
> Issue Type: Bug
> Components: clients, consumer
> Affects Versions: 0.10.2.1, 1.0.0
> Reporter: HongLiang
> Labels: patch
> Attachments: _01704061-1bed-41cc-90ea-794fe955815a.png,
> _1085b605-f8f7-4df8-a13f-e0dd855179e4.png,
> _883ddf50-beb7-4e87-9630-168acaa9b046.png, auto-commit-fail-bugs.patch,
> e6cf53be-e128-47dc-a45a-79439a9e55ff.png,
> pool_46ba3275-7b56-4c64-a4f4-7280eb7f1728.png
>
>
> Auto-commit of offsets fail, and not recover forever. at
> sendOffsetCommitRequest, while "generation equal NULL", ConsumerCoordinator
> request will fail always. it maybe a bug. error log below:
> has more and more warn log ....
> "2017-12-01 22:08:39.112 WARN pool-390-thread-1#1
> (ConsumerCoordinator.java:626) - Auto-commit of offsets
> {drawing_gift_sent-1=OffsetAndMetadata{offset=32150359, metadata=''}} failed
> for group gift_rich_audience_write: Commit cannot be completed since the
> group has already rebalanced and assigned the partitions to another member.
> This means that the time between subsequent calls to poll() was longer than
> the configured max.poll.interval.ms, which typically implies that the poll
> loop is spending too much time message processing. You can address this
> either by increasing the session timeout or by reducing the maximum size of
> batches returned in poll() with max.poll.records."
> !e6cf53be-e128-47dc-a45a-79439a9e55ff.png|thumbnail!
> !_883ddf50-beb7-4e87-9630-168acaa9b046.png|thumbnail!
> !pool_46ba3275-7b56-4c64-a4f4-7280eb7f1728.png|thumbnail!
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)