Browse Source

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>
pull/8674/head
zhaohaidao 5 years ago committed by GitHub
parent
commit
f50bd5f80e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 5
      clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
  2. 7
      clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java

5
clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java

@ -1015,8 +1015,9 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> { @@ -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 {

7
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java

@ -282,6 +282,13 @@ public class KafkaConsumerTest { @@ -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();

Loading…
Cancel
Save