[ 
https://issues.apache.org/jira/browse/KAFKA-16010?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17798772#comment-17798772
 ] 

Philip Nee commented on KAFKA-16010:
------------------------------------

I thought KAFKA-16026 (fixing the max poll) would fix this; however, I'm 
getting quite a strange error. See below: Seems like some cases we need to 
reset the HeartbeatState: due to RebalanceTimeoutMs must be provided in first 
request.

 
{code:java}
[2023-12-19 15:07:37,732] WARN [QuorumController id=1000] Performing controller 
activation. The metadata log appears to be empty. Appending 1 bootstrap 
record(s) in metadata transaction at metadata.version 3.8-IV0 from bootstrap 
source 'test harness'. Setting the ZK migration state to NONE since this is a 
de-novo KRaft cluster. (org.apache.kafka.controller.QuorumController:108)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,835] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] GroupHeartbeatRequest failed due to error: INVALID_REQUEST 
(org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:380)
[2023-12-19 15:07:39,836] ERROR [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] Member T9t83LZLS9-GxOGCX1sI9A with epoch 0 transitioned to 
FATAL state 
(org.apache.kafka.clients.consumer.internals.MembershipManagerImpl:487)
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] WARN [Consumer clientId=ConsumerTestConsumer, 
groupId=my-test] An error occurred when processing the event: 
RebalanceTimeoutMs must be provided in first request. 
(org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer:193)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.
[2023-12-19 15:07:39,875] ERROR [daemon-consumer-assignment]: Error due to 
(kafka.api.AbstractConsumerTest$ConsumerAssignmentPoller:139)
org.apache.kafka.common.errors.InvalidRequestException: RebalanceTimeoutMs must 
be provided in first request.Did not get valid assignment for partitions 
HashSet(topic1-2, topic1-4, topic-1, topic-0, topic1-5, topic1-1, topic1-0, 
topic1-3). Instead, got ArrayBuffer(Set(topic1-1, topic1-0, topic1-2), Set(), 
Set(topic1-3, topic-1))
org.opentest4j.AssertionFailedError: Did not get valid assignment for 
partitions HashSet(topic1-2, topic1-4, topic-1, topic-0, topic1-5, topic1-1, 
topic1-0, topic1-3). Instead, got ArrayBuffer(Set(topic1-1, topic1-0, 
topic1-2), Set(), Set(topic1-3, topic-1))
 {code}
 

 

> Fix PlaintextConsumerTest.testMultiConsumerSessionTimeoutOnStopPolling
> ----------------------------------------------------------------------
>
>                 Key: KAFKA-16010
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16010
>             Project: Kafka
>          Issue Type: Test
>          Components: clients, consumer, unit tests
>    Affects Versions: 3.7.0
>            Reporter: Kirk True
>            Priority: Major
>              Labels: consumer-threading-refactor, kip-848
>             Fix For: 3.8.0
>
>
> The integration test 
> {{PlaintextConsumerTest.testMultiConsumerSessionTimeoutOnStopPolling}} is 
> failing when using the {{AsyncKafkaConsumer}}.
> The error is:
> {code}
> org.opentest4j.AssertionFailedError: Did not get valid assignment for 
> partitions [topic1-2, topic1-4, topic-1, topic-0, topic1-5, topic1-1, 
> topic1-0, topic1-3] after one consumer left
>       at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:38)
>       at org.junit.jupiter.api.Assertions.fail(Assertions.java:134)
>       at 
> kafka.api.AbstractConsumerTest.validateGroupAssignment(AbstractConsumerTest.scala:286)
>       at 
> kafka.api.PlaintextConsumerTest.runMultiConsumerSessionTimeoutTest(PlaintextConsumerTest.scala:1883)
>       at 
> kafka.api.PlaintextConsumerTest.testMultiConsumerSessionTimeoutOnStopPolling(PlaintextConsumerTest.scala:1281)
> {code}
> The logs include these lines:
>  
> {code}
> [2023-12-13 15:26:40,736] WARN [Consumer clientId=ConsumerTestConsumer, 
> groupId=my-test] consumer poll timeout has expired. This means 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 processing messages. You can address this either by increasing 
> max.poll.interval.ms or by reducing the maximum size of batches returned in 
> poll() with max.poll.records. 
> (org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:188)
> [2023-12-13 15:26:40,736] WARN [Consumer clientId=ConsumerTestConsumer, 
> groupId=my-test] consumer poll timeout has expired. This means 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 processing messages. You can address this either by increasing 
> max.poll.interval.ms or by reducing the maximum size of batches returned in 
> poll() with max.poll.records. 
> (org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:188)
> [2023-12-13 15:26:40,736] WARN [Consumer clientId=ConsumerTestConsumer, 
> groupId=my-test] consumer poll timeout has expired. This means 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 processing messages. You can address this either by increasing 
> max.poll.interval.ms or by reducing the maximum size of batches returned in 
> poll() with max.poll.records. 
> (org.apache.kafka.clients.consumer.internals.HeartbeatRequestManager:188)
> {code} 
> I don't know if that's related or not.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to