kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mj...@apache.org
Subject [kafka] branch trunk updated: KAFKA-9984 Should fail the subscription when pattern is empty (#8665)
Date Thu, 14 May 2020 21:18:52 GMT
This is an automated email from the ASF dual-hosted git repository.

mjsax pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new f50bd5f  KAFKA-9984 Should fail the subscription when pattern is empty (#8665)
f50bd5f is described below

commit f50bd5f80e33598e7fd70290f0c0e809443bc9a0
Author: zhaohaidao <zhaohaidao2008@hotmail.com>
AuthorDate: Fri May 15 05:18:15 2020 +0800

    KAFKA-9984 Should fail the subscription when pattern is empty (#8665)
    
    Reviewers: Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>,
Matthias J. Sax <matthias@confluent.io>
---
 .../main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 5 +++--
 .../java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java  | 7 +++++++
 2 files changed, 10 insertions(+), 2 deletions(-)

diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 5b0a8c7..81d286f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -1015,8 +1015,9 @@ public class KafkaConsumer<K, V> implements Consumer<K, V>
{
     @Override
     public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) {
         maybeThrowInvalidGroupIdException();
-        if (pattern == null)
-            throw new IllegalArgumentException("Topic pattern to subscribe to cannot be null");
+        if (pattern == null || pattern.toString().equals(""))
+            throw new IllegalArgumentException("Topic pattern to subscribe to cannot be "
+ (pattern == null ?
+                    "null" : "empty"));
 
         acquireAndEnsureOpen();
         try {
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index 410eec3..56f6975 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -282,6 +282,13 @@ public class KafkaConsumerTest {
         }
     }
 
+    @Test(expected = IllegalArgumentException.class)
+    public void testSubscriptionOnEmptyPattern() {
+        try (KafkaConsumer<byte[], byte[]> consumer = newConsumer(groupId)) {
+            consumer.subscribe(Pattern.compile(""));
+        }
+    }
+
     @Test(expected = IllegalStateException.class)
     public void testSubscriptionWithEmptyPartitionAssignment() {
         Properties props = new Properties();


Mime
View raw message