[ https://issues.apache.org/jira/browse/KAFKA-5882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16169964#comment-16169964 ]
Seweryn Habdank-Wojewodzki edited comment on KAFKA-5882 at 9/18/17 12:51 PM: ----------------------------------------------------------------------------- Scenario for Kafka (0.11.0.0): 1. Kafka broker with 2 nodes (active) + 3 Zookeeper nodes are working. 2. One instance of the streaming app works. 3. Start the second instance of the streaming app on another mashine. 4. Settings: 1 stream thread. 5. Kafka start up: {code} 2017-09-18 14:13:16 INFO AppInfoParser:83 - Kafka version : 0.11.0.0 2017-09-18 14:13:16 INFO AppInfoParser:84 - Kafka commitId : cb8625948210849f 2017-09-18 14:13:16 DEBUG KafkaProducer:410 - Kafka producer started 2017-09-18 14:13:16 DEBUG StateDirectory:135 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] Acquired state dir lock for task 13_5 2017-09-18 14:13:16 DEBUG Sender:157 - Starting Kafka producer I/O thread. 2017-09-18 14:13:16 INFO ProcessorStateManager:122 - task [13_5] Created state store manager for task 13_5 with the acquired state dir lock 2017-09-18 14:13:17 DEBUG Metrics:403 - Added sensor with name commit 2017-09-18 14:13:17 DEBUG Metrics:403 - Added sensor with name 13_5-commit 2017-09-18 14:13:17 INFO StreamThread:1248 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] Created active task 13_5 with assigned partitions [a0291_topic-5] 2017-09-18 14:13:17 INFO StreamThread:193 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] partition assignment took 68 ms. current active tasks: [] current standby tasks: [] 2017-09-18 14:13:17 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-18 14:13:17 DEBUG ConsumerCoordinator:826 - Group streamer fetching committed offsets for partitions: [gai34_topic-5, gai34_topic-9, int06_topic-8, gai34_topic-7, int77_topic-8, a0291_topic-6, a0291_topic-8, gai10_topic-9, int06_topic-3, gai10_topic-7, int06_topic-5, gai10_topic- 5, int62_topic-8, gai10_topic-3, gai34_topic-4, gai34_topic-8, int06_topic-7, c0737_topic-9, gai34_topic-6, int06_topic-9, int77_topic-7, a0291_topic-9, a0291_topic-5, gai10_topic-8, a0291_topic-7, int06_topic-4, gai10_topic-6, int06_topic-6, gai10_topic-4, int62_topic-9, int62_topic-7, in t77_topic-9] {code} This week I will intensively test Kafka 0.11.0.1, perhaps it will be better. was (Author: habdank): Scenario for Kafka (0.11.0.0): 1. Kafka broker with 2 nodes (active) + 3 Zookeeper nodes are working. 2. One instance of the streaming app works. 3. Start second instance of the streaming app on another mashine. 4. Settings: 1 stream thread. 5. Kafka start up: {code} 2017-09-18 14:13:16 INFO AppInfoParser:83 - Kafka version : 0.11.0.0 2017-09-18 14:13:16 INFO AppInfoParser:84 - Kafka commitId : cb8625948210849f 2017-09-18 14:13:16 DEBUG KafkaProducer:410 - Kafka producer started 2017-09-18 14:13:16 DEBUG StateDirectory:135 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] Acquired state dir lock for task 13_5 2017-09-18 14:13:16 DEBUG Sender:157 - Starting Kafka producer I/O thread. 2017-09-18 14:13:16 INFO ProcessorStateManager:122 - task [13_5] Created state store manager for task 13_5 with the acquired state dir lock 2017-09-18 14:13:17 DEBUG Metrics:403 - Added sensor with name commit 2017-09-18 14:13:17 DEBUG Metrics:403 - Added sensor with name 13_5-commit 2017-09-18 14:13:17 INFO StreamThread:1248 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] Created active task 13_5 with assigned partitions [a0291_topic-5] 2017-09-18 14:13:17 INFO StreamThread:193 - stream-thread [streamer-923cc4d5-22aa-40f2-95ff-0fbda292b346-StreamThread-1] partition assignment took 68 ms. current active tasks: [] current standby tasks: [] 2017-09-18 14:13:17 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-18 14:13:17 DEBUG ConsumerCoordinator:826 - Group streamer fetching committed offsets for partitions: [gai34_topic-5, gai34_topic-9, int06_topic-8, gai34_topic-7, int77_topic-8, a0291_topic-6, a0291_topic-8, gai10_topic-9, int06_topic-3, gai10_topic-7, int06_topic-5, gai10_topic- 5, int62_topic-8, gai10_topic-3, gai34_topic-4, gai34_topic-8, int06_topic-7, c0737_topic-9, gai34_topic-6, int06_topic-9, int77_topic-7, a0291_topic-9, a0291_topic-5, gai10_topic-8, a0291_topic-7, int06_topic-4, gai10_topic-6, int06_topic-6, gai10_topic-4, int62_topic-9, int62_topic-7, in t77_topic-9] {code} This week I will intensively test Kafka 0.11.0.1, perhaps it will be better. > 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 > Reporter: Seweryn Habdank-Wojewodzki > > 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 (v6.4.14#64029)