[ https://issues.apache.org/jira/browse/KAFKA-5882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16345527#comment-16345527 ]
Guozhang Wang commented on KAFKA-5882: -------------------------------------- I looked into this issue a bit more and here are my two cents: 1. KAFKA-6260 itself should not be related to this issue, since it is a regression that gets introduced only in 1.0.0 and has been fixed in the coming release of 1.0.1 and 1.1.0; on the other hand, you reported this issue in 0.11.0.0 / 0.11.0.1 already. 2. You reported in 0.11.0.2 it is not crashing any more, however the error log you pasted indicate that there is still a fatal error, this error should still cause Streams to stop unless it is a ProducerFencedException which means you have enabled the EOS feature. So to help further investigating this issue, could you elaborate a bit more on the following: 1. On 1.0.0, when it crashes, could you share the log stack trace that which exception gets thrown? Is it still the same NPE as in the description of this JIRA? 2. On 0.11.0.2, could you also share the full log snippet around the time when the ERROR log is reported? > NullPointerException in StreamTask > ---------------------------------- > > Key: KAFKA-5882 > URL: https://issues.apache.org/jira/browse/KAFKA-5882 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 0.11.0.0, 0.11.0.1, 0.11.0.2 > Reporter: Seweryn Habdank-Wojewodzki > Priority: Major > > It seems bugfix [KAFKA-5073|https://issues.apache.org/jira/browse/KAFKA-5073] > is made, but introduce some other issue. > In some cases (I am not sure which ones) I got NPE (below). > I would expect that even in case of FATAL error anythink except NPE is thrown. > {code} > 2017-09-12 23:34:54 ERROR ConsumerCoordinator:269 - User provided listener > org.apache.kafka.streams.processor.internals.StreamThread$RebalanceListener > for group streamer failed on partition assignment > java.lang.NullPointerException: null > at > org.apache.kafka.streams.processor.internals.StreamTask.<init>(StreamTask.java:123) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.createStreamTask(StreamThread.java:1234) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread$TaskCreator.createTask(StreamThread.java:294) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:254) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.addStreamTasks(StreamThread.java:1313) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.access$1100(StreamThread.java:73) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread$RebalanceListener.onPartitionsAssigned(StreamThread.java:183) > ~[myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:265) > [myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:363) > [myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:310) > [myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:297) > [myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1078) > [myapp-streamer.jar:?] > at > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1043) > [myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:582) > [myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:553) > [myapp-streamer.jar:?] > at > org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:527) > [myapp-streamer.jar:?] > 2017-09-12 23:34:54 INFO StreamThread:1040 - stream-thread > [streamer-3a44578b-faa8-4b5b-bbeb-7a7f04639563-StreamThread-1] Shutting down > 2017-09-12 23:34:54 INFO KafkaProducer:972 - Closing the Kafka producer with > timeoutMillis = 9223372036854775807 ms. > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)