kafka-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Guozhang Wang (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (KAFKA-4539) StreamThread is not correctly creating StandbyTasks
Date Fri, 16 Dec 2016 00:51:58 GMT

     [ https://issues.apache.org/jira/browse/KAFKA-4539?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Guozhang Wang updated KAFKA-4539:
---------------------------------
    Description: 
Fails because {{createStandbyTask(..)}} can return null if the topology for the {{TaskId}}
doesn't have any state stores.

{code}
[2016-12-14 12:20:29,758] ERROR [StreamThread-1] User provided listener org.apache.kafka.streams.processor.internals.StreamThread$1
for group kafka-music-charts failed on partition assignment (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
java.lang.NullPointerException
	at org.apache.kafka.streams.processor.internals.StreamThread$StandbyTaskCreator.createTask(StreamThread.java:1241)
	at org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:1188)
	at org.apache.kafka.streams.processor.internals.StreamThread.addStandbyTasks(StreamThread.java:915)
	at org.apache.kafka.streams.processor.internals.StreamThread.access$400(StreamThread.java:72)
	at org.apache.kafka.streams.processor.internals.StreamThread$1.onPartitionsAssigned(StreamThread.java:239)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:230)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:314)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:278)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:261)
	at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1022)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:987)
	at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:568)
	at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:357)
{code}

Also fails because the checkpointedOffsets from the newly created {{StandbyTask}} aren't added
to the offsets map, so the partitions don't get assigned. We then get:


  was:
Fails because {{createStandbyTask(..)}} can return null fi the topology for the {{TaskId}}
doesn't have any state stores.

{code}
[2016-12-14 12:20:29,758] ERROR [StreamThread-1] User provided listener org.apache.kafka.streams.processor.internals.StreamThread$1
for group kafka-music-charts failed on partition assignment (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
java.lang.NullPointerException
	at org.apache.kafka.streams.processor.internals.StreamThread$StandbyTaskCreator.createTask(StreamThread.java:1241)
	at org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:1188)
	at org.apache.kafka.streams.processor.internals.StreamThread.addStandbyTasks(StreamThread.java:915)
	at org.apache.kafka.streams.processor.internals.StreamThread.access$400(StreamThread.java:72)
	at org.apache.kafka.streams.processor.internals.StreamThread$1.onPartitionsAssigned(StreamThread.java:239)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:230)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:314)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:278)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:261)
	at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1022)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:987)
	at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:568)
	at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:357)
{code}

Also fails because the checkpointedOffsets from the newly created {{StandbyTask}} aren't added
to the offsets map, so the partitions don't get assigned. We then get:



> StreamThread is not correctly creating  StandbyTasks
> ----------------------------------------------------
>
>                 Key: KAFKA-4539
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4539
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.10.2.0
>            Reporter: Damian Guy
>            Assignee: Damian Guy
>             Fix For: 0.10.2.0
>
>
> Fails because {{createStandbyTask(..)}} can return null if the topology for the {{TaskId}}
doesn't have any state stores.
> {code}
> [2016-12-14 12:20:29,758] ERROR [StreamThread-1] User provided listener org.apache.kafka.streams.processor.internals.StreamThread$1
for group kafka-music-charts failed on partition assignment (org.apache.kafka.clients.consumer.internals.ConsumerCoordinator)
> java.lang.NullPointerException
> 	at org.apache.kafka.streams.processor.internals.StreamThread$StandbyTaskCreator.createTask(StreamThread.java:1241)
> 	at org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:1188)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.addStandbyTasks(StreamThread.java:915)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.access$400(StreamThread.java:72)
> 	at org.apache.kafka.streams.processor.internals.StreamThread$1.onPartitionsAssigned(StreamThread.java:239)
> 	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:230)
> 	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:314)
> 	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:278)
> 	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:261)
> 	at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1022)
> 	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:987)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:568)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:357)
> {code}
> Also fails because the checkpointedOffsets from the newly created {{StandbyTask}} aren't
added to the offsets map, so the partitions don't get assigned. We then get:



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message