[ https://issues.apache.org/jira/browse/KAFKA-4362?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Joel Koshy updated KAFKA-4362: ------------------------------ Summary: Consumer can fail after reassignment of the offsets topic partition (was: Offset commits fail after a partition reassignment) Yes definitely an issue, so I'm updating the title. So the reassignment of the offsets topic will perpetually cause offset commits to fail. A new consumer joining the group will talk to the new coordinator and incorrectly becomes an isolated group. Any rebalance of the remaining instances of the actual group (that's still talking to the old coordinator) can hit this error and die: {code} [2016-11-01 15:37:56,120] WARN Auto offset commit failed for group testgroup: Unexpected error in commit: The server experienced an unexpected error when processing the request (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator) ... <kill another instance in the group to force a rebalance> ... [2016-11-01 15:37:56,120] INFO Revoking previously assigned partitions [testtopic-0, testtopic-1] for group testgroup (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator) [2016-11-01 15:37:56,120] INFO (Re-)joining group testgroup (org.apache.kafka.clients.consumer.internals.AbstractCoordinator) [2016-11-01 15:37:56,124] ERROR Error processing message, terminating consumer process: (kafka.tools.ConsoleConsumer$) org.apache.kafka.common.KafkaException: Unexpected error from SyncGroup: The server experienced an unexpected error when processing the request at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$SyncGroupResponseHandler.handle(AbstractCoordinator.java:518) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$SyncGroupResponseHandler.handle(AbstractCoordinator.java:485) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:742) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:722) at org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:186) at org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:149) at org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:116) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.fireCompletion(ConsumerNetworkClient.java:479) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.firePendingCompletedRequests(ConsumerNetworkClient.java:316) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:256) at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:180) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:308) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:277) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:259) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1013) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:979) at kafka.consumer.NewShinyConsumer.receive(BaseConsumer.scala:100) at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:120) at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:75) at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:50) at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala) {code} > Consumer can fail after reassignment of the offsets topic partition > ------------------------------------------------------------------- > > Key: KAFKA-4362 > URL: https://issues.apache.org/jira/browse/KAFKA-4362 > Project: Kafka > Issue Type: Bug > Affects Versions: 0.10.1.0 > Reporter: Joel Koshy > Assignee: Jiangjie Qin > > When a consumer offsets topic partition reassignment completes, an offset > commit shows this: > {code} > java.lang.IllegalArgumentException: Message format version for partition 100 > not found > at > kafka.coordinator.GroupMetadataManager$$anonfun$14.apply(GroupMetadataManager.scala:633) > ~[kafka_2.10.jar:?] > at > kafka.coordinator.GroupMetadataManager$$anonfun$14.apply(GroupMetadataManager.scala:633) > ~[kafka_2.10.jar:?] > at scala.Option.getOrElse(Option.scala:120) ~[scala-library-2.10.4.jar:?] > at > kafka.coordinator.GroupMetadataManager.kafka$coordinator$GroupMetadataManager$$getMessageFormatVersionAndTimestamp(GroupMetadataManager.scala:632) > ~[kafka_2.10.jar:?] > at > ... > {code} > The issue is that the replica has been deleted so the > {{GroupMetadataManager.getMessageFormatVersionAndTimestamp}} throws this > exception instead which propagates as an unknown error. > Unfortunately consumers don't respond to this and will fail their offset > commits. > One workaround in the above situation is to bounce the cluster - the consumer > will be forced to rediscover the group coordinator. > (Incidentally, the message incorrectly prints the number of partitions > instead of the actual partition.) -- This message was sent by Atlassian JIRA (v6.3.4#6332)