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

Jason Gustafson commented on KAFKA-4801:
----------------------------------------

After my patch was merged, this seems to now be showing up as this:
{code}
kafka.api.ConsumerBounceTest > testConsumptionWithBrokerFailures FAILED
    org.apache.kafka.clients.consumer.CommitFailedException: 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.
        at 
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:776)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.handle(ConsumerCoordinator.java:722)
        at 
org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:797)
        at 
org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:778)
        at 
org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:204)
        at 
org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:167)
        at 
org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:127)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:486)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:346)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:260)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:206)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:182)
        at 
org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:589)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1104)
        at 
kafka.api.ConsumerBounceTest.consumeWithBrokerFailures(ConsumerBounceTest.scala:113)
        at 
kafka.api.ConsumerBounceTest.testConsumptionWithBrokerFailures(ConsumerBounceTest.scala:84)
{code}
This suggests that the consumer is falling out of the group in the test case 
for whatever reason.

> Transient test failure (part 2): 
> ConsumerBounceTest.testConsumptionWithBrokerFailures
> -------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4801
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4801
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Armin Braun
>            Assignee: Jason Gustafson
>            Priority: Minor
>              Labels: transient-system-test-failure
>
> There is still some (but very little ... when reproducing this you need more 
> than 100 runs in half the cases statistically) instability left in the test
> {code}
> ConsumerBounceTest.testConsumptionWithBrokerFailures
> {code}
> Resulting in this exception being thrown at a relatively low rate (I'd say 
> def less than 0.5% of all runs on my machine).
> {code}
> kafka.api.ConsumerBounceTest > testConsumptionWithBrokerFailures FAILED
>     java.lang.IllegalArgumentException: You can only check the position for 
> partitions assigned to this consumer.
>         at 
> org.apache.kafka.clients.consumer.KafkaConsumer.position(KafkaConsumer.java:1271)
>         at 
> kafka.api.ConsumerBounceTest.consumeWithBrokerFailures(ConsumerBounceTest.scala:96)
>         at 
> kafka.api.ConsumerBounceTest.testConsumptionWithBrokerFailures(ConsumerBounceTest.scala:69)
> {code}
> this was also reported in a comment to the original KAFKA-4198
> https://issues.apache.org/jira/browse/KAFKA-4198?focusedCommentId=15765468&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15765468



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to