[ https://issues.apache.org/jira/browse/KAFKA-7044?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16608368#comment-16608368 ]
ASF GitHub Bot commented on KAFKA-7044: --------------------------------------- apovzner opened a new pull request #5627: KAFKA-7044: Fix Fetcher.fetchOffsetsByTimes and NPE in describe consumer group URL: https://github.com/apache/kafka/pull/5627 `kafka-consumer-groups --describe --group ...` can result in NullPointerException for two reasons: 1) Fetcher.fetchOffsetsByTimes() may return too early, without sending list offsets request for topic partitions that are not in cached metadata. 2) `ConsumerGroupCommand.getLogEndOffsets()` and `getLogStartOffsets()` assumed that endOffsets()/beginningOffsets() which eventually call Fetcher.fetchOffsetsByTimes(), would return a map with all the topic partitions passed to endOffsets()/beginningOffsets() and that values are not null. Because of #1, null values were possible if some of the topic partitions were already known (in metadata cache) and some not (metadata cache did not have entries for some of the topic partitions). However, even with fixing #1, endOffsets()/beginningOffsets() may return a map with some topic partitions missing, when list offset request returns a non-retriable error. This happens in corner cases such as message format on broker is before 0.10, or maybe in cases of some other errors. Testing: -- added unit test to verify fix in Fetcher.fetchOffsetsByTimes() -- did some manual testing with `kafka-consumer-groups --describe`, causing NPE. Was not able to reproduce any NPE cases with DescribeConsumerGroupTest.scala, ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on 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 > kafka-consumer-groups.sh NullPointerException describing round robin or > sticky assignors > ---------------------------------------------------------------------------------------- > > Key: KAFKA-7044 > URL: https://issues.apache.org/jira/browse/KAFKA-7044 > Project: Kafka > Issue Type: Bug > Components: tools > Affects Versions: 1.1.0 > Environment: CentOS 7.4, Oracle JDK 1.8 > Reporter: Jeff Field > Assignee: Vahid Hashemian > Priority: Major > Fix For: 2.1.0 > > > We've recently moved to using the round robin assignor for one of our > consumer groups, and started testing the sticky assignor. In both cases, > using Kafka 1.1.0 we get a null pointer exception *unless* the group being > described is rebalancing: > {code:java} > kafka-consumer-groups --bootstrap-server fqdn:9092 --describe --group > groupname-for-consumer > Error: Executing consumer group command failed due to null > [2018-06-12 01:32:34,179] DEBUG Exception in consumer group command > (kafka.admin.ConsumerGroupCommand$) > java.lang.NullPointerException > at scala.Predef$.Long2long(Predef.scala:363) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$getLogEndOffsets$2.apply(ConsumerGroupCommand.scala:612) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$getLogEndOffsets$2.apply(ConsumerGroupCommand.scala:610) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at > scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) > at scala.collection.immutable.List.foreach(List.scala:392) > at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) > at scala.collection.immutable.List.map(List.scala:296) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService.getLogEndOffsets(ConsumerGroupCommand.scala:610) > at > kafka.admin.ConsumerGroupCommand$ConsumerGroupService$class.describePartitions(ConsumerGroupCommand.scala:328) > at > kafka.admin.ConsumerGroupCommand$ConsumerGroupService$class.collectConsumerAssignment(ConsumerGroupCommand.scala:308) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService.collectConsumerAssignment(ConsumerGroupCommand.scala:544) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$10$$anonfun$13.apply(ConsumerGroupCommand.scala:571) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$10$$anonfun$13.apply(ConsumerGroupCommand.scala:565) > at scala.collection.immutable.List.flatMap(List.scala:338) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$10.apply(ConsumerGroupCommand.scala:565) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService$$anonfun$10.apply(ConsumerGroupCommand.scala:558) > at scala.Option.map(Option.scala:146) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService.collectGroupOffsets(ConsumerGroupCommand.scala:558) > at > kafka.admin.ConsumerGroupCommand$ConsumerGroupService$class.describeGroup(ConsumerGroupCommand.scala:271) > at > kafka.admin.ConsumerGroupCommand$KafkaConsumerGroupService.describeGroup(ConsumerGroupCommand.scala:544) > at kafka.admin.ConsumerGroupCommand$.main(ConsumerGroupCommand.scala:77) > at kafka.admin.ConsumerGroupCommand.main(ConsumerGroupCommand.scala) > [2018-06-12 01:32:34,255] DEBUG Removed sensor with name connections-closed: > (org.apache.kafka.common.metrics.Metrics){code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)