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 72C66200B40 for ; Fri, 1 Jul 2016 20:09:13 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 7172C160A61; Fri, 1 Jul 2016 18:09:13 +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 B8C6E160A6C for ; Fri, 1 Jul 2016 20:09:12 +0200 (CEST) Received: (qmail 50972 invoked by uid 500); 1 Jul 2016 18:09:11 -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 50757 invoked by uid 99); 1 Jul 2016 18:09:11 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 01 Jul 2016 18:09:11 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 3F0B12C02A2 for ; Fri, 1 Jul 2016 18:09:11 +0000 (UTC) Date: Fri, 1 Jul 2016 18:09:11 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (KAFKA-3854) Subsequent regex subscription calls fail MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 01 Jul 2016 18:09:13 -0000 [ https://issues.apache.org/jira/browse/KAFKA-3854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15359384#comment-15359384 ] ASF GitHub Bot commented on KAFKA-3854: --------------------------------------- GitHub user vahidhashemian reopened a pull request: https://github.com/apache/kafka/pull/1572 KAFKA-3854: Fix issues with new consumer's subsequent regex (pattern) subscriptions This patch fixes two issues: 1. Subsequent regex subscriptions fail with the new consumer. 2. Subsequent regex subscriptions would not immediately refresh metadata to change the subscription of the new consumer and trigger a rebalance. The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat `subscribe()` to the same pattern or some wait time until the next automatic metadata refresh would handle that case. An integration test was also added to verify these issues are fixed with this PR. You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka KAFKA-3854 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/1572.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #1572 ---- commit 5fb472a48a0231aece798d601c3427bd058fc078 Author: Vahid Hashemian Date: 2016-06-16T21:19:32Z KAFKA-3854: Fix issues with new consumer's subsequent regex (pattern) subscriptions This patch fixes two issues: 1. Subsequent regex subscriptions fail with the new consumer. 2. Subsequent regex subscriptions would not actually refresh metadata and change the subscription of the new consumer nor they would trigger a rebalance. The final note on the JIRA stating that a later created topic that matches a consumer's subscription pattern would not be assigned to the consumer upon creation seems to be as designed. A repeat subscribe() to the same pattern would be needed to handle that case. Unit tests for regex subscriptions will be handled in KAFKA-3897. ---- > Subsequent regex subscription calls fail > ---------------------------------------- > > Key: KAFKA-3854 > URL: https://issues.apache.org/jira/browse/KAFKA-3854 > Project: Kafka > Issue Type: Bug > Components: consumer > Reporter: Vahid Hashemian > Assignee: Vahid Hashemian > > There are a couple of issues with regex subscription in the new consumer: > h6. Subsequent Subscriptions Fail > When consecutive calls are made to new consumer's [regex subscription|https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L850], like below: > {code} > consumer.subscribe(Pattern.compile("..."), listener); > consumer.poll(0); > consumer.subscribe(Pattern.compile("f.."), listener); > consumer.poll(0); > {code} > the second call fails with the following error: > {code} > Exception in thread "main" java.lang.IllegalStateException: Subscription to topics, partitions and pattern are mutually exclusive > at org.apache.kafka.clients.consumer.internals.SubscriptionState.subscribe(SubscriptionState.java:175) > at org.apache.kafka.clients.consumer.KafkaConsumer.subscribe(KafkaConsumer.java:854) > ... > {code} > h6. Subsequent Subscriptions Fail to Trigger a Subscription Change and Rebalance > Even after the code is tweaked to get around the above issue, only the first call to regex subscription triggers an actual subscription and the subscription's rebalance listener. The reason is the regex {{subscription}} / {{poll}} does not directly call {{changeSubscription(...)}} in which {{needsPartitionAssignment}} is set and causes a rebalance. This method is called only during the first regex {{subscription}} / {{poll}} when coordinator is unknown and [a {{client.poll}} call|https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java#L179] is made which eventually leads to [a {{changeSubscription}} call|https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L161]. The subsequent call does not reach this point because the coordinator is already known. > It seems due to the same reason, if a consumer is subscribed to a pattern, and later on a new topic is created that matches that pattern, the consumer does not become subscribed it. -- This message was sent by Atlassian JIRA (v6.3.4#6332)