You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Jason Gustafson (Jira)" <ji...@apache.org> on 2020/12/18 23:57:00 UTC

[jira] [Updated] (KAFKA-10870) Consumer should handle REBALANCE_IN_PROGRESS from JoinGroup

     [ https://issues.apache.org/jira/browse/KAFKA-10870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jason Gustafson updated KAFKA-10870:
------------------------------------
    Description: 
We hit a timeout when persisting group metadata to the __consumer_offsets topic:
{code}
[2020-12-18 18:06:08,209] DEBUG [GroupMetadataManager brokerId=1] Metadata from group test_group_id with generation 1 failed when appending to log due to org.apache.kafka.common.errors.TimeoutException (kafka.coordinator.group.GroupMetadataManager)
[2020-12-18 18:06:08,210] WARN [GroupCoordinator 1]: Failed to persist metadata for group test_group_id: The group is rebalancing, so a rejoin is needed. (kafka.coordinator.group.GroupCoordinator)
{code}

This in turn resulted in a REBALANCE_IN_PROGRESS being returned from the JoinGroup:

{code}
[2020-12-18 18:06:08,211] INFO Completed request:RequestHeader(apiKey=JOIN_GROUP, apiVersion=7, clientId=consumer-test_group_id-test_group_id-instance-1, correlationId=3) -- {group_id=test_group_id,session_timeout_ms=60000,rebalance_timeout_ms=300000,member_id=,group_instance_id=test_group_id-instance-1,protocol_type=consumer,protocols=[{name=range,metadata=java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],_tagged_fields={}}],_tagged_fields={}},response:{throttle_time_ms=0,error_code=27,generation_id=1,protocol_type=consumer,protocol_name=range,leader=test_group_id-instance-2-32e72316-2c3f-40d6-bc34-8ec23d633d34,member_id=,members=[],_tagged_fields={}} from connection 172.31.46.222:9092-172.31.44.169:41310-6;totalTime:5014.825,requestQueueTime:0.193,localTime:11.575,remoteTime:5002.195,throttleTime:0.66,responseQueueTime:0.105,sendTime:0.094,sendIoTime:0.038,securityProtocol:PLAINTEXT,principal:User:ANONYMOUS,listener:PLAINTEXT,clientInformation:ClientInformation(softwareName=apache-kafka-java, softwareVersion=5.5.3-ce) (kafka.request.logger)      
{code}

The consumer has no logic to handle REBALANCE_IN_PROGRESS from JoinGroup.

{code}
[2020-12-18 18:06:08,210] ERROR [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Attempt to join group failed due to unexpected error
: The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2020-12-18 18:06:08,211] INFO [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Join group failed with org.apache.kafka.common.KafkaE
xception: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2020-12-18 18:06:08,211] ERROR Error during processing, terminating consumer process:  (org.apache.kafka.tools.VerifiableConsumer)
org.apache.kafka.common.KafkaException: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed.
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:653)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:574)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1096)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1076)
        at org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
{code}

  was:
We hit a timeout when persisting group metadata to the __consumer_offsets topic:
{code}
[2020-12-18 18:06:07,889] DEBUG Created a new incremental FetchContext for session id 58555532, epoch 53: added 0 partition(s), updated 0 partition(s), removed 0 partition(s) (kafka.server.FetchManager)
[2020-12-18 18:06:08,209] DEBUG [GroupMetadataManager brokerId=1] Metadata from group test_group_id with generation 1 failed when appending to log due to org.apache.kafka.common.errors.TimeoutException (kafka.coordinator.group.GroupMetadataManager)
[2020-12-18 18:06:08,210] WARN [GroupCoordinator 1]: Failed to persist metadata for group test_group_id: The group is rebalancing, so a rejoin is needed. (kafka.coordinator.group.GroupCoordinator)
{code}

This in turn resulted in a REBALANCE_IN_PROGRESS being returned from the JoinGroup:

{code}
[2020-12-18 18:06:08,211] INFO Completed request:RequestHeader(apiKey=JOIN_GROUP, apiVersion=7, clientId=consumer-test_group_id-test_group_id-instance-1, correlationId=3) -- {group_id=test_group_id,session_timeout_ms=60000,rebalance_timeout_ms=300000,member_id=,group_instance_id=test_group_id-instance-1,protocol_type=consumer,protocols=[{name=range,metadata=java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],_tagged_fields={}}],_tagged_fields={}},response:{throttle_time_ms=0,error_code=27,generation_id=1,protocol_type=consumer,protocol_name=range,leader=test_group_id-instance-2-32e72316-2c3f-40d6-bc34-8ec23d633d34,member_id=,members=[],_tagged_fields={}} from connection 172.31.46.222:9092-172.31.44.169:41310-6;totalTime:5014.825,requestQueueTime:0.193,localTime:11.575,remoteTime:5002.195,throttleTime:0.66,responseQueueTime:0.105,sendTime:0.094,sendIoTime:0.038,securityProtocol:PLAINTEXT,principal:User:ANONYMOUS,listener:PLAINTEXT,clientInformation:ClientInformation(softwareName=apache-kafka-java, softwareVersion=5.5.3-ce) (kafka.request.logger)      
{code}

The consumer has no logic to handle REBALANCE_IN_PROGRESS from JoinGroup.

{code}
[2020-12-18 18:06:08,210] ERROR [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Attempt to join group failed due to unexpected error
: The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2020-12-18 18:06:08,211] INFO [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Join group failed with org.apache.kafka.common.KafkaE
xception: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
[2020-12-18 18:06:08,211] ERROR Error during processing, terminating consumer process:  (org.apache.kafka.tools.VerifiableConsumer)
org.apache.kafka.common.KafkaException: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed.
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:653)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:574)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1096)
        at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1076)
        at org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
{code}


> Consumer should handle REBALANCE_IN_PROGRESS from JoinGroup
> -----------------------------------------------------------
>
>                 Key: KAFKA-10870
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10870
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Jason Gustafson
>            Priority: Major
>
> We hit a timeout when persisting group metadata to the __consumer_offsets topic:
> {code}
> [2020-12-18 18:06:08,209] DEBUG [GroupMetadataManager brokerId=1] Metadata from group test_group_id with generation 1 failed when appending to log due to org.apache.kafka.common.errors.TimeoutException (kafka.coordinator.group.GroupMetadataManager)
> [2020-12-18 18:06:08,210] WARN [GroupCoordinator 1]: Failed to persist metadata for group test_group_id: The group is rebalancing, so a rejoin is needed. (kafka.coordinator.group.GroupCoordinator)
> {code}
> This in turn resulted in a REBALANCE_IN_PROGRESS being returned from the JoinGroup:
> {code}
> [2020-12-18 18:06:08,211] INFO Completed request:RequestHeader(apiKey=JOIN_GROUP, apiVersion=7, clientId=consumer-test_group_id-test_group_id-instance-1, correlationId=3) -- {group_id=test_group_id,session_timeout_ms=60000,rebalance_timeout_ms=300000,member_id=,group_instance_id=test_group_id-instance-1,protocol_type=consumer,protocols=[{name=range,metadata=java.nio.HeapByteBuffer[pos=0 lim=26 cap=26],_tagged_fields={}}],_tagged_fields={}},response:{throttle_time_ms=0,error_code=27,generation_id=1,protocol_type=consumer,protocol_name=range,leader=test_group_id-instance-2-32e72316-2c3f-40d6-bc34-8ec23d633d34,member_id=,members=[],_tagged_fields={}} from connection 172.31.46.222:9092-172.31.44.169:41310-6;totalTime:5014.825,requestQueueTime:0.193,localTime:11.575,remoteTime:5002.195,throttleTime:0.66,responseQueueTime:0.105,sendTime:0.094,sendIoTime:0.038,securityProtocol:PLAINTEXT,principal:User:ANONYMOUS,listener:PLAINTEXT,clientInformation:ClientInformation(softwareName=apache-kafka-java, softwareVersion=5.5.3-ce) (kafka.request.logger)      
> {code}
> The consumer has no logic to handle REBALANCE_IN_PROGRESS from JoinGroup.
> {code}
> [2020-12-18 18:06:08,210] ERROR [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Attempt to join group failed due to unexpected error
> : The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2020-12-18 18:06:08,211] INFO [Consumer instanceId=test_group_id-instance-1, clientId=consumer-test_group_id-test_group_id-instance-1, groupId=test_group_id] Join group failed with org.apache.kafka.common.KafkaE
> xception: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed. (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)
> [2020-12-18 18:06:08,211] ERROR Error during processing, terminating consumer process:  (org.apache.kafka.tools.VerifiableConsumer)
> org.apache.kafka.common.KafkaException: Unexpected error in join group response: The group is rebalancing, so a rejoin is needed.
>         at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:653)
>         at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:574)
>         at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1096)
>         at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:1076)
>         at org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)