[ https://issues.apache.org/jira/browse/KAFKA-4905?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16146985#comment-16146985 ]
Fredrik Vraalsen commented on KAFKA-4905: ----------------------------------------- Hi! We've just stumbled upon this exception in our Kafka Streams app (using 0.11), but with a different use case: We have a streams application that has been running for a while, and we tried to deploy a new version where we had removed one of the subtopologies. This was using the Processor API so had it's own input and output topics that were produced and consumed by the rest of the streams app. However, now we're getting exceptions on startup: {code} Uncaught exception: Thread xyz-StreamThread-1 stopped unexpectedly after Assigned partition foo-1 for non-subscribed topic regex pattern; subscription pattern is bar java.lang.IllegalArgumentException: Assigned partition foo-1 for non-subscribed topic regex pattern; subscription pattern is bar at org.apache.kafka.clients.consumer.internals.SubscriptionState.assignFromSubscribed(SubscriptionState.java:195) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:226) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:363) at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:310) at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:297) at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1078) at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1043) at org.apache.kafka.streams.processor.internals.StreamThread.pollRequests(StreamThread.java:582) at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:553) at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:527) {code} Is there any way to work around this? Or is it currently not possible to *remove* things from a Kafka Streams topology? > StreamPartitionAssignor doesn't respect subscriptions to assign partitions. > --------------------------------------------------------------------------- > > Key: KAFKA-4905 > URL: https://issues.apache.org/jira/browse/KAFKA-4905 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 0.10.2.0 > Reporter: Florian Hussonnois > > Both RangeAssignor and RoundRobinAssignor use the subscriptions to assign > partition to each consumer. This allow to have two consumers belonging to the > the same group and subscribing to two differents topics. > This doesn't seem to be the case of the StreamPartitionAssignor resulting to > an IllegalArgumentException thrown during rebalance. > java.lang.IllegalArgumentException: Assigned partition foo-2 for > non-subscribed topic regex pattern; subscription pattern is bar > at > org.apache.kafka.clients.consumer.internals.SubscriptionState.assignFromSubscribed(SubscriptionState.java:190) > at > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:216) > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:352) > at > org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:303) > This is because the consumer group leader attempt to assign partitions to a > consumer that didn't subscribe to the associated topic. -- This message was sent by Atlassian JIRA (v6.4.14#64029)