sorry for the previous empty msg. Jay's idea should work. basically, we override the close method in Serializer interface.
public interface Serializer<T> extends Closeable { @Override public void close(); } On Tue, Apr 28, 2015 at 1:10 PM, Steven Wu <stevenz...@gmail.com> wrote: > > > On Tue, Apr 28, 2015 at 1:03 PM, Ewen Cheslack-Postava <e...@confluent.io> > wrote: > >> Good point Jay. More specifically we were already implementing without the >> checked exception, we'd need to override close() in the Serializer and >> Deserializer interfaces and omit the throws clause. That definitely makes >> them source compatible. Not sure about binary compatibility, I couldn't >> find a quick answer but I think it's probably still compatible. >> >> -Ewen >> >> On Tue, Apr 28, 2015 at 12:30 PM, Jay Kreps <jay.kr...@gmail.com> wrote: >> >> > Hey guys, >> > >> > You can implement Closable without the checked exception. Having close() >> > methods throw checked exceptions isn't very useful unless there is a way >> > for the caller to recover. In this case there really isn't, right? >> > >> > -Jay >> > >> > On Mon, Apr 27, 2015 at 5:51 PM, Guozhang Wang <wangg...@gmail.com> >> wrote: >> > >> > > Folks, >> > > >> > > In a recent commit I made regarding KAFKA-2121, there is an omitted >> API >> > > change which makes Serializer / Deserializer extending from Closeable, >> > > whose close() call could throw IOException by declaration. Hence now >> some >> > > scenario like: >> > > >> > > --------------------- >> > > >> > > Serializer<T> keySerializer = ... >> > > Serializer<T> valueSerializer = ... >> > > KafkaProducer producer = new KafkaProducer(config, keySerializer, >> > > valueSerializer) >> > > // ... >> > > keySerializer.close() >> > > valueSerializer.close() >> > > >> > > --------------------- >> > > >> > > will need to capture IOException now. >> > > >> > > Want to bring this up for people's attention, and you opinion on >> whether >> > we >> > > should revert this change? >> > > >> > > -- Guozhang >> > > >> > >> >> >> >> -- >> Thanks, >> Ewen >> > >