hachikuji commented on a change in pull request #8934: URL: https://github.com/apache/kafka/pull/8934#discussion_r450978217
########## File path: clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java ########## @@ -1218,17 +1218,20 @@ public void assign(Collection<TopicPartition> partitions) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - // poll for new data until the timeout expires do { client.maybeTriggerWakeup(); if (includeMetadataInTimeout) { - // try to update assignment metadata BUT do not need to block on the timer, - // since even if we are 1) in the middle of a rebalance or 2) have partitions - // with unknown starting positions we may still want to return some data - // as long as there are some partitions fetchable; NOTE we always use a timer with 0ms - // to never block on completing the rebalance procedure if there's any - updateAssignmentMetadataIfNeeded(time.timer(0L)); + // try to update assignment metadata BUT do not need to block on the timer if we still have Review comment: I did a simple experiment with one broker and one consumer. After joining the group, I did a kill -9 to stop the broker. I had the consumer log the poll timeout that was used in `pollForFetches`. What I saw is this: ``` [2020-07-07 08:53:10,825] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,825] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,825] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,825] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,839] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,839] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,839] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) [2020-07-07 08:53:10,839] INFO [Consumer clientId=consumer-foo-1, groupId=foo] Polling for fetches with timeout 0 (org.apache.kafka.clients.consumer.KafkaConsumer) ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org