Closeable is part of `java.io` while AutoClosable is part of
`java.lang`. Thus, the second one is more generic. Also, JavaDoc points
out that `Closable#close()` must be idempotent while
`AutoClosable#close()` can have side effects.

Thus, I am not sure atm which one suits better.

However, it's a good hint, that `AutoClosable#close()` declares `throws
Exception` and thus, it seems to be a backward incompatible change.
Hence, I am not sure if we can actually move forward easily with this KIP.

Nit: `RecordCollectorImpl` is an internal class that implements
`RecordCollector` -- should `RecordCollector extends AutoCloseable`?


-Matthias


On 9/27/18 7:46 PM, Chia-Ping Tsai wrote:
>> (Although I am not quite sure
>> when one is more desirable than the other)
> 
> Most kafka's classes implementing Closeable/AutoCloseable doesn't throw 
> checked exception in close() method. Perhaps we should have a 
> "KafkaCloseable" interface which has a close() method without throwing any 
> checked exception...
> 
> On 2018/09/27 19:11:20, Yishun Guan <gyis...@gmail.com> wrote: 
>> Hi All,
>>
>> Chia-Ping, I agree, similar to VarifiableConsumer, VarifiableProducer
>> should be implementing Closeable as well (Although I am not quite sure
>> when one is more desirable than the other), also I just looked through
>> your list - these are some great additions, I will add them to the
>> list.
>>
>> Thanks,
>> Yishun
>> On Thu, Sep 27, 2018 at 3:26 AM Dongjin Lee <dong...@apache.org> wrote:
>>>
>>> Hi Yishun,
>>>
>>> Thank you for your great KIP. In fact, I have also encountered the cases
>>> where Autoclosable is so desired several times! Let me inspect more
>>> candidate classes as well.
>>>
>>> +1. I also refined your KIP a little bit.
>>>
>>> Best,
>>> Dongjin
>>>
>>> On Thu, Sep 27, 2018 at 12:21 PM Chia-Ping Tsai <chia7...@apache.org> wrote:
>>>
>>>> hi Yishun
>>>>
>>>> Thanks for nice KIP!
>>>>
>>>> Q1)
>>>> Why VerifiableProducer extend Closeable rather than AutoCloseable?
>>>>
>>>> Q2)
>>>> I grep project and then noticed there are other close methods but do not
>>>> implement AutoCloseable.
>>>> For example:
>>>> 1) WorkerConnector
>>>> 2) MemoryRecordsBuilder
>>>> 3) MetricsReporter
>>>> 4) ExpiringCredentialRefreshingLogin
>>>> 5) KafkaChannel
>>>> 6) ConsumerInterceptor
>>>> 7) SelectorMetrics
>>>> 8) HeartbeatThread
>>>>
>>>> Cheers,
>>>> Chia-Ping
>>>>
>>>>
>>>> On 2018/09/26 23:44:31, Yishun Guan <gyis...@gmail.com> wrote:
>>>>> Hi All,
>>>>>
>>>>> Here is a trivial KIP:
>>>>>
>>>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=93325308
>>>>>
>>>>> Suggestions are welcome.
>>>>>
>>>>> Thanks,
>>>>> Yishun
>>>>>
>>>>
>>>
>>>
>>> --
>>> *Dongjin Lee*
>>>
>>> *A hitchhiker in the mathematical world.*
>>>
>>> *github:  <http://goog_969573159/>github.com/dongjinleekr
>>> <http://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/dongjinleekr
>>> <http://kr.linkedin.com/in/dongjinleekr>slideshare:
>>> www.slideshare.net/dongjinleekr
>>> <http://www.slideshare.net/dongjinleekr>*
>>

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to