[ https://issues.apache.org/jira/browse/KAFKA-5430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16055418#comment-16055418 ]
Lior Chaga commented on KAFKA-5430: ----------------------------------- Hi, I gathered some more information regarding this issue. The cluster is deployed on 2 adjacent data centers (with fiber network between them). Brokers 9,12,13,14,15,16 are on data center A. Brokers 5,6,7,8,10,11 are on data center B. There was a firmware upgrade for Arista network devices at DC A. During the upgrade, there were errors in the broker logs: {code} [2017-06-04 04:33:03,836] WARN [ReplicaFetcherThread-0-12], Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@145b009c (kafka.server.ReplicaFetcherThread) java.io.IOException: Connection to 12 was disconnected before the response was read at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:114) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1$$anonfun$apply$1.apply(NetworkClientBlockingOps.scala:112) at scala.Option.foreach(Option.scala:236) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:112) at kafka.utils.NetworkClientBlockingOps$$anonfun$blockingSendAndReceive$extension$1.apply(NetworkClientBlockingOps.scala:108) at kafka.utils.NetworkClientBlockingOps$.recursivePoll$1(NetworkClientBlockingOps.scala:136) at kafka.utils.NetworkClientBlockingOps$.kafka$utils$NetworkClientBlockingOps$$pollContinuously$extension(NetworkClientBlockingOps.scala:142) at kafka.utils.NetworkClientBlockingOps$.blockingSendAndReceive$extension(NetworkClientBlockingOps.scala:108) at kafka.server.ReplicaFetcherThread.sendRequest(ReplicaFetcherThread.scala:249) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:234) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {code} And these: {code} [2017-06-04 04:33:13,579] WARN [ReplicaFetcherThread-0-13], Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@4abe0742 (kafka.server.ReplicaFetcherThread) java.io.IOException: Connection to broker013:6667 (id: 13 rack: null) failed at kafka.utils.NetworkClientBlockingOps$.awaitReady$1(NetworkClientBlockingOps.scala:84) at kafka.utils.NetworkClientBlockingOps$.blockingReady$extension(NetworkClientBlockingOps.scala:94) at kafka.server.ReplicaFetcherThread.sendRequest(ReplicaFetcherThread.scala:244) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:234) at kafka.server.ReplicaFetcherThread.fetch(ReplicaFetcherThread.scala:42) at kafka.server.AbstractFetcherThread.processFetchRequest(AbstractFetcherThread.scala:118) at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:103) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:63) {code} The latter seemed (didn't go through all broker logs) to appear only on brokers from DC A (in which the upgrade was performed). Hope this is helpful to understand the issue. > new consumers getting data for revoked partitions > ------------------------------------------------- > > Key: KAFKA-5430 > URL: https://issues.apache.org/jira/browse/KAFKA-5430 > Project: Kafka > Issue Type: Bug > Components: clients, consumer > Affects Versions: 0.10.2.0 > Reporter: Lior Chaga > Attachments: kafka_trace.log.gz > > > Due to bad configuration applied to network components, we experienced issues > with communication between kafka brokers (causing under replication) as well > as producers/consumers not being able to work against kafka. > The symptoms on the consumer were many errors of the following form: > {code} > 2017-06-04 04:27:35,200 ERROR [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] TaboolaKafkaConsumer [] - Failed > committing to kafka topicPartitions > [requestlegacy-2,requestlegacy-0,requestlegacy-1] > org.apache.kafka.clients.consumer.RetriableCommitFailedException: Offset > commit failed with a retriable exception. You should retry committing offsets. > Caused by: org.apache.kafka.common.errors.DisconnectException > {code} > So far so good. However, upon network recovery, there were several rebalance > operations, which eventually resulted in only one consumer (#14) being > assigned with all topic partitions (at this case we're talking about a > consumer groups for which all consumers are running in same process): > {code} > 2017-06-04 04:27:02,168 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-8, requestlegacy-9] > for group session_parser_02 > 2017-06-04 04:27:04,208 INFO [Kafka Topics Cosumer > requestlegacy.consumer-15_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-10, requestlegacy-11] > for group session_parser_02 > 2017-06-04 04:27:18,167 INFO [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-3, requestlegacy-4, > requestlegacy-5] for group session_parser_02 > 2017-06-04 04:27:20,232 INFO [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-2, requestlegacy-0, > requestlegacy-1] for group session_parser_02 > 2017-06-04 04:27:20,236 INFO [Kafka Topics Cosumer > requestlegacy.consumer-15_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-9, requestlegacy-10, > requestlegacy-11] for group session_parser_02 > 2017-06-04 04:27:20,237 INFO [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-3, requestlegacy-4, requestlegacy-5] > for group session_parser_02 > 2017-06-04 04:27:20,237 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-6, requestlegacy-7, requestlegacy-8] > for group session_parser_02 > 2017-06-04 04:27:20,332 INFO [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-2, requestlegacy-0, requestlegacy-1] > for group session_parser_02 > 2017-06-04 04:28:52,368 INFO [Kafka Topics Cosumer > requestlegacy.consumer-13_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-6, requestlegacy-7] > for group session_parser_02 > 2017-06-04 04:29:15,201 INFO [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-2, requestlegacy-0, > requestlegacy-1] for group session_parser_02 > 2017-06-04 04:30:22,379 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-6, requestlegacy-7, > requestlegacy-8] for group session_parser_02 > 2017-06-04 04:30:24,431 INFO [Kafka Topics Cosumer > requestlegacy.consumer-15_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-9, requestlegacy-10, > requestlegacy-11] for group session_parser_02 > 2017-06-04 04:30:38,229 INFO [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-3, requestlegacy-4, > requestlegacy-5] for group session_parser_02 > 2017-06-04 04:30:39,156 INFO [Kafka Topics Cosumer > requestlegacy.consumer-13_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-6, requestlegacy-7] for group > session_parser_02 > 2017-06-04 04:30:39,156 INFO [Kafka Topics Cosumer > requestlegacy.consumer-15_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-10, requestlegacy-11] for group > session_parser_02 > 2017-06-04 04:30:39,156 INFO [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-2, requestlegacy-0, requestlegacy-1] > for group session_parser_02 > 2017-06-04 04:30:39,156 INFO [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-3, requestlegacy-4, requestlegacy-5] > for group session_parser_02 > 2017-06-04 04:30:39,157 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-8, requestlegacy-9] for group > session_parser_02 > 2017-06-04 04:49:51,118 INFO [Kafka Topics Cosumer > requestlegacy.consumer-13_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-6, requestlegacy-7] > for group session_parser_02 > 2017-06-04 04:50:40,847 INFO [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-3, requestlegacy-4, > requestlegacy-5] for group session_parser_02 > 2017-06-04 04:50:58,898 INFO [Kafka Topics Cosumer > requestlegacy.consumer-15_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-10, requestlegacy-11] > for group session_parser_02 > 2017-06-04 04:50:59,010 INFO [Kafka Topics Cosumer > requestlegacy.consumer-11_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-2, requestlegacy-0, > requestlegacy-1] for group session_parser_02 > 2017-06-04 04:52:39,732 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-8, requestlegacy-9] > for group session_parser_02 > 2017-06-04 04:55:57,633 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-8, requestlegacy-9] for group > session_parser_02 > 2017-06-04 04:57:37,662 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-8, requestlegacy-9] > for group session_parser_02 > 2017-06-04 05:00:57,601 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-2, requestlegacy-3, requestlegacy-4, > requestlegacy-5, requestlegacy-6, requestlegacy-7, requestlegacy-8, > requestlegacy-9, requestlegacy-0, requestlegacy-1, requestlegacy-10, > requestlegacy-11] for group session_parser_02 > 2017-06-04 05:02:37,605 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - > Revoking previously assigned partitions [requestlegacy-2, requestlegacy-3, > requestlegacy-4, requestlegacy-5, requestlegacy-6, requestlegacy-7, > requestlegacy-8, requestlegacy-9, requestlegacy-0, requestlegacy-1, > requestlegacy-10, requestlegacy-11] for group session_parser_02 > 2017-06-04 05:02:37,666 INFO [Kafka Topics Cosumer > requestlegacy.consumer-14_session_parser_02] ConsumerCoordinator [] - Setting > newly assigned partitions [requestlegacy-2, requestlegacy-3, requestlegacy-4, > requestlegacy-5, requestlegacy-6, requestlegacy-7, requestlegacy-8, > requestlegacy-9, requestlegacy-0, requestlegacy-1, requestlegacy-10, > requestlegacy-11] for group session_parser_02 > {code} > The interesting part is that after the last rebalance, other consumers were > still polling messages for hours, and failed upon commit (notice the revoke > message on consumer-12, and the commit failure below on same consumer, 2 > minutes later): > {code} > 2017-06-04 04:52:41,396 ERROR [Kafka Topics Cosumer > requestlegacy.consumer-12_session_parser_02] TaboolaKafkaConsumer [] - Failed > committing to kafka topicPartitions > [requestlegacy-3,requestlegacy-4,requestlegacy-5] > 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. > {code} > To complete the picture from kafka side, kafka-consumer-group reports all > partitions assigned to consumer-14: > {code} > requestlegacy 0 9092336 9092337 1 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 1 9094923 9094923 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 2 9091382 9091382 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 3 9097052 9097052 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 4 9097542 9097542 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 5 9098915 9098915 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 6 9098308 9098309 1 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 7 9096984 9096984 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 8 9100548 9100548 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 9 9097615 9097615 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 10 9095553 9095553 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > requestlegacy 11 9089520 9089520 0 > consumer-14-aafabf03-2e94-42ef-922a-ba699c5a8517 /10.105.137.28 > consumer-14 > {code} > We took a trace log (attached) of org.apache.kafka for our consuming service > when we discovered the issue, prior to restarting the service (which resulted > in full recovery). > Also, this happened for multiple consumer groups, this is just one example. -- This message was sent by Atlassian JIRA (v6.4.14#64029)