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

Terry Beard commented on KAFKA-14565:
-------------------------------------

Hey [~ChrisEgerton] thanks for the feedback.   Yes, I did consider altering 
{{{}AbstractConfig::getConfiguredInstances{}}}, but I was thinking we should 
not alter that logic  unless there is a defect which I do not believe this to 
be the case.  Also without knowing the extent of the logic rewrite/test/impact 
to other client classes, I was attempting to find an approach which isolates 
the change to the most relevant classes:
 * Kafka Consumer/Producer client
 * Kafka Consumer/Producer Interceptor
 * Kafka Consumer/Producer Interceptors container 



while also providing a clearer interceptor interface definition which addresses 
resource management concerns and decouples creation/configuration concerns.  To 
that end,  adding a default open method with no implementation seemed like the 
best approach as it
 # Should not require any action on the part developers
 # We can explicitly compliment and better leverage existing Consumer/Producer 
client {{close }}logic with open{{ }}logic
 # This approach is suggested for future compatibility in 
[KIP-42|http://example.com/]: 



{quote}
h1. Compatibility, Deprecation, and Migration Plan

It will not impact any of existing clients. When clients upgrade to new 
version, they do not need to add _interceptor.classes_ config.

*Future compatibility.* When/if new methods will be added to 
ProducerInterceptor and ConsumerInterceptor (as part of other KIP(s)), they 
will be added with an empty implementation to the Producer/ConsumerInterceptor 
interfaces. This is a new feature in Java 8.
{quote}
 

 

> Improve Interceptor Resource Leakage Prevention
> -----------------------------------------------
>
>                 Key: KAFKA-14565
>                 URL: https://issues.apache.org/jira/browse/KAFKA-14565
>             Project: Kafka
>          Issue Type: Improvement
>          Components: clients
>            Reporter: Terry Beard
>            Assignee: Terry Beard
>            Priority: Major
>             Fix For: 3.5.0
>
>
> The Consumer and Producer interceptor interfaces and their corresponding 
> Kafka Consumer and Producer constructors do not adequately support cleanup of 
> underlying interceptor resources. 
> Currently within the Kafka Consumer and Kafka Producer constructors,  the 
> AbstractConfig.getConfiguredInstances()  is delegated responsibility for both 
> creating and configuring each interceptor listed in the interceptor.classes 
> property and returns a configured  List<ConsumerInterceptor<K,V>> 
> interceptors.
> This dual responsibility for both creation and configuration is problematic 
> when it involves multiple interceptors where at least one interceptor's 
> configure method implementation creates and/or depends on objects which 
> creates threads, connections or other resources which requires clean up and 
> the subsequent interceptor's configure method raises a runtime exception.  
> This raising of the runtime exception produces a resource leakage in the 
> first interceptor as the interceptor container i.e. 
> ConsumerInterceptors/ProducerInterceptors is never created and therefore the 
> first interceptor's and really any interceptor's close method are never 
> called.  
> To help ensure the respective container interceptors are able to invoke their 
> respective interceptor close methods for proper resource clean up, I propose 
> defining a default open method with no implementation and check exception on 
> the respective Consumer/Producer interceptor interfaces.  This open method 
> will be responsible for creating threads and/or objects which utilizes 
> threads, connections or other resource which requires clean up.  
> Additionally, the default open method enables implementation optionality as 
> it's empty default behavior means it will do nothing when unimplemented.  
> Additionally, the Kafka Consumer/Producer Interceptor containers will 
> implement a corresponding maybeOpen method which throws a checked exception.  
> In order to maintain backwards compatibility with earlier developed 
> interceptors the maybeOpen will check whether the interceptor's interface 
> contains the newer open method before calling it accordingly.   



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to