[ 
https://issues.apache.org/jira/browse/FLINK-10774?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16714813#comment-16714813
 ] 

ASF GitHub Bot commented on FLINK-10774:
----------------------------------------

tillrohrmann commented on a change in pull request #7020: [FLINK-10774] [Kafka] 
connection leak when partition discovery is disabled an…
URL: https://github.com/apache/flink/pull/7020#discussion_r240235652
 
 

 ##########
 File path: 
flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBase.java
 ##########
 @@ -756,6 +756,20 @@ public void cancel() {
                        // the discovery loop may currently be sleeping 
in-between
                        // consecutive discoveries; interrupt to shutdown faster
                        discoveryLoopThread.interrupt();
+               } else {
+                       // when partition discovery is disabled,
+                       // we should still call the close method.
+                       // otherwise we may have connection leak,
+                       // if open method throws an exception after 
partitionDiscoverer is constructed.
+                       // In this case, run method won't be executed
+                       // and partitionDiscoverer.close() won't be called.
+                       if (partitionDiscoverer != null) {
+                               try {
+                                       partitionDiscoverer.close();
+                               } catch (Exception e) {
+                                       LOG.error("failed to close 
partitionDiscoverer", e);
+                               }
+                       }
 
 Review comment:
   I think we should not close the `partitionDiscoverer` here. Instead we 
should close it in the `open` method if no longer needed and in the `run` 
method when leaving the `kafkaFetcher` thread.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> connection leak when partition discovery is disabled and open throws exception
> ------------------------------------------------------------------------------
>
>                 Key: FLINK-10774
>                 URL: https://issues.apache.org/jira/browse/FLINK-10774
>             Project: Flink
>          Issue Type: Bug
>          Components: Kafka Connector
>    Affects Versions: 1.4.2, 1.5.5, 1.6.2
>            Reporter: Steven Zhen Wu
>            Assignee: Steven Zhen Wu
>            Priority: Major
>              Labels: pull-request-available
>
> Here is the scenario to reproduce the issue
>  * partition discovery is disabled
>  * open method throws an exception (e.g. when broker SSL authorization denies 
> request)
> In this scenario, run method won't be executed. As a result, 
> _partitionDiscoverer.close()_ won't be called. that caused the connection 
> leak, because KafkaConsumer is initialized but not closed. That has caused 
> outage that brought down our Kafka cluster, when a high-parallelism job got 
> into a restart/failure loop.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to