[
https://issues.apache.org/jira/browse/FLINK-8497?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16456316#comment-16456316
]
ASF GitHub Bot commented on FLINK-8497:
---------------------------------------
GitHub user alexpf opened a pull request:
https://github.com/apache/flink/pull/5929
[FLINK-8497] [connectors] KafkaConsumer throws NPE if topic doesn't exist
## What is the purpose of the change
To make the Kafka connector behavior clearer for the API user, by throwing
a specific exception
## Brief change log
- Added throwing IllegalStateException when KafkaConsumer returns null
for the topic partitions.
- Since Kafka maintains the map of topics, returning null means topic
does not exist.
## Verifying this change
This change added tests and can be verified as follows:
- Added new test class Kafka09PartitionDiscovererTest.java with an
appropriate unit test
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): no
- The public API, i.e., is any changed class annotated with
`@Public(Evolving)`: no
- The serializers: no
- The runtime per-record code paths (performance sensitive): no
- Anything that affects deployment or recovery: JobManager (and its
components), Checkpointing, Yarn/Mesos, ZooKeeper: no
- The S3 file system connector: no
## Documentation
- Does this pull request introduce a new feature? no
- If yes, how is the feature documented? n/a
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/alexpf/flink FLINK-8497
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/5929.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 #5929
----
commit 6441cc568d4a4ff632c3f7daf351e2d11f339af3
Author: Aleksei Lesnov <alexey.lesnov@...>
Date: 2018-04-27T12:24:22Z
[FLINK-8497] [connectors] KafkaConsumer throws NPE if topic doesn't exist
----
> KafkaConsumer throws NPE if topic doesn't exist
> -----------------------------------------------
>
> Key: FLINK-8497
> URL: https://issues.apache.org/jira/browse/FLINK-8497
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector
> Affects Versions: 1.4.0
> Reporter: chris snow
> Assignee: Aleksei Lesnov
> Priority: Minor
>
> If I accidentally set the kafka consumer with a topic that doesn't exist:
> {code:java}
> FlinkKafkaConsumer011 kafkaConsumer = new FlinkKafkaConsumer011(
> "does_not_exist",
> new JSONKeyValueDeserializationSchema(false),
> properties
> );
> DataStream<String> input = env.addSource(kafkaConsumer);{code}
> Flink throws NPE
> {code:java}
> Caused by: java.lang.NullPointerException
> at
> org.apache.flink.streaming.connectors.kafka.internal.Kafka09PartitionDiscoverer.getAllPartitionsForTopics(Kafka09PartitionDiscoverer.java:75)
> at
> org.apache.flink.streaming.connectors.kafka.internals.AbstractPartitionDiscoverer.discoverPartitions(AbstractPartitionDiscoverer.java:128)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.open(FlinkKafkaConsumerBase.java:415)
> at
> org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36)
> at
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.open(AbstractUdfStreamOperator.java:102)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.openAllOperators(StreamTask.java:393)
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:254)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
> at java.lang.Thread.run(Thread.java:748){code}
> Maybe Flink could through an IllegalStateException("Topic not found")?
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)