Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E0431200BF0 for ; Fri, 16 Dec 2016 01:52:00 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id DEF55160B15; Fri, 16 Dec 2016 00:52:00 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 0FDEC160B30 for ; Fri, 16 Dec 2016 01:51:59 +0100 (CET) Received: (qmail 99572 invoked by uid 500); 16 Dec 2016 00:51:58 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 99544 invoked by uid 99); 16 Dec 2016 00:51:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 16 Dec 2016 00:51:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 808972C03E6 for ; Fri, 16 Dec 2016 00:51:58 +0000 (UTC) Date: Fri, 16 Dec 2016 00:51:58 +0000 (UTC) From: "Guozhang Wang (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (KAFKA-4539) StreamThread is not correctly creating StandbyTasks MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 16 Dec 2016 00:52:01 -0000 [ 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)