Hi Guozhang, I had an offline discussion with Matthias and Bill about it. It is thought that `to` offers some benefit, i.e., syntactic sugar, so perhaps no harm in keeping it. However, `through` less so, seeing as we can materialize stores via `filter`, `map` etc, so one of the main benefits of `through` no longer exists. WDYT?
Thanks, Damian On Tue, 12 Sep 2017 at 18:17 Guozhang Wang <wangg...@gmail.com> wrote: > Hi Damian, > > Why we are deprecating KTable.through while keeping KTable.to? Should we > either keep both of them or deprecate both of them in favor or > KTable.toStream if people agree that it is confusing to users? > > > Guozhang > > > On Tue, Sep 12, 2017 at 1:18 AM, Damian Guy <damian....@gmail.com> wrote: > > > Hi All, > > > > A minor update to the KIP, i needed to add KTable.to(Produced) for > > consistency. KTable.through will be deprecated in favour of using > > KTable.toStream().through() > > > > Thanks, > > Damian > > > > On Thu, 7 Sep 2017 at 08:52 Damian Guy <damian....@gmail.com> wrote: > > > > > Thanks all. The vote is now closed and the KIP has been accepted with: > > > 2 non binding votes - bill and matthias > > > 3 binding - Damian, Guozhang, Sriram > > > > > > Regards, > > > Damian > > > > > > On Tue, 5 Sep 2017 at 22:24 Sriram Subramanian <r...@confluent.io> > wrote: > > > > > >> +1 > > >> > > >> On Tue, Sep 5, 2017 at 1:33 PM, Guozhang Wang <wangg...@gmail.com> > > wrote: > > >> > > >> > +1 > > >> > > > >> > On Fri, Sep 1, 2017 at 3:45 PM, Matthias J. Sax < > > matth...@confluent.io> > > >> > wrote: > > >> > > > >> > > +1 > > >> > > > > >> > > On 9/1/17 2:53 PM, Bill Bejeck wrote: > > >> > > > +1 > > >> > > > > > >> > > > On Thu, Aug 31, 2017 at 10:20 AM, Damian Guy < > > damian....@gmail.com> > > >> > > wrote: > > >> > > > > > >> > > >> Thanks everyone for voting! Unfortunately i've had to make a > bit > > >> of an > > >> > > >> update based on some issues found during implementation. > > >> > > >> The main changes are: > > >> > > >> BytesStoreSupplier -> StoreSupplier > > >> > > >> Addition of: > > >> > > >> WindowBytesStoreSupplier, KeyValueBytesStoreSupplier, > > >> > > >> SessionBytesStoreSupplier that will restrict store types to > > <Bytes, > > >> > > byte[]> > > >> > > >> 3 new overloads added to Materialized to enable developers to > > >> create a > > >> > > >> Materialized of the appropriate type, i..e, WindowStore etc > > >> > > >> Update DSL where Materialized is used such that the stores have > > >> > generic > > >> > > >> types of <Bytes, byte[]> > > >> > > >> Some minor changes to the arguments to > > Store#persistentWindowStore > > >> and > > >> > > >> Store#persistentSessionStore > > >> > > >> > > >> > > >> Please take a look and recast the votes. > > >> > > >> > > >> > > >> Thanks for your time, > > >> > > >> Damian > > >> > > >> > > >> > > >> On Fri, 25 Aug 2017 at 17:05 Matthias J. Sax < > > >> matth...@confluent.io> > > >> > > >> wrote: > > >> > > >> > > >> > > >>> Thanks Damian. Great KIP! > > >> > > >>> > > >> > > >>> +1 > > >> > > >>> > > >> > > >>> > > >> > > >>> -Matthias > > >> > > >>> > > >> > > >>> On 8/25/17 6:45 AM, Damian Guy wrote: > > >> > > >>>> Hi, > > >> > > >>>> > > >> > > >>>> I've just realised we need to add two methods to > > >> StateStoreBuilder > > >> > or > > >> > > >> it > > >> > > >>>> isn't going to work: > > >> > > >>>> > > >> > > >>>> Map<String, String> logConfig(); > > >> > > >>>> boolean loggingEnabled(); > > >> > > >>>> > > >> > > >>>> These are needed when we are building the topology and > > >> determining > > >> > > >>>> changelog topic names and configs. > > >> > > >>>> > > >> > > >>>> > > >> > > >>>> I've also update the KIP to add > > >> > > >>>> > > >> > > >>>> StreamBuilder#stream(String topic) > > >> > > >>>> > > >> > > >>>> StreamBuilder#stream(String topic, Consumed options) > > >> > > >>>> > > >> > > >>>> > > >> > > >>>> Thanks > > >> > > >>>> > > >> > > >>>> > > >> > > >>>> On Thu, 24 Aug 2017 at 22:11 Sriram Subramanian < > > >> r...@confluent.io> > > >> > > >>> wrote: > > >> > > >>>> > > >> > > >>>>> +1 > > >> > > >>>>> > > >> > > >>>>> On Thu, Aug 24, 2017 at 10:20 AM, Guozhang Wang < > > >> > wangg...@gmail.com> > > >> > > >>>>> wrote: > > >> > > >>>>> > > >> > > >>>>>> +1. Thanks Damian! > > >> > > >>>>>> > > >> > > >>>>>> On Thu, Aug 24, 2017 at 9:47 AM, Bill Bejeck < > > >> bbej...@gmail.com> > > >> > > >>> wrote: > > >> > > >>>>>> > > >> > > >>>>>>> Thanks for the KIP! > > >> > > >>>>>>> > > >> > > >>>>>>> +1 > > >> > > >>>>>>> > > >> > > >>>>>>> Thanks, > > >> > > >>>>>>> Bill > > >> > > >>>>>>> > > >> > > >>>>>>> On Thu, Aug 24, 2017 at 12:25 PM, Damian Guy < > > >> > damian....@gmail.com > > >> > > > > > >> > > >>>>>> wrote: > > >> > > >>>>>>> > > >> > > >>>>>>>> Hi, > > >> > > >>>>>>>> > > >> > > >>>>>>>> I'd like to kick off the voting thread for KIP-182: > > >> > > >>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP- > > >> > > >>>>>>>> 182%3A+Reduce+Streams+DSL+overloads+and+allow+easier+ > > >> > > >>>>>>>> use+of+custom+storage+engines > > >> > > >>>>>>>> > > >> > > >>>>>>>> Thanks, > > >> > > >>>>>>>> Damian > > >> > > >>>>>>>> > > >> > > >>>>>>> > > >> > > >>>>>> > > >> > > >>>>>> > > >> > > >>>>>> > > >> > > >>>>>> -- > > >> > > >>>>>> -- Guozhang > > >> > > >>>>>> > > >> > > >>>>> > > >> > > >>>> > > >> > > >>> > > >> > > >>> > > >> > > >> > > >> > > > > > >> > > > > >> > > > > >> > > > >> > > > >> > -- > > >> > -- Guozhang > > >> > > > >> > > > > > > > > > -- > -- Guozhang >