Thanks Luke for driving the KIP discussion as well as preparing the POC!
The POC PR is especially useful for me to get all the implementation devils
in the details :)


Guozhang

On Thu, Feb 3, 2022 at 7:04 PM Luke Chen <show...@gmail.com> wrote:

> Hi Matthias and all,
>
> Thanks for your comments.
> I've updated the KIP to remove the Stores API change.
>
> And I'll also close the vote for this KIP.
> The KIP is accepted with 3 binding votes (Matthias, Guozhang, and John).
>
> Thank you very much.
>
> Luke
>
> On Tue, Feb 1, 2022 at 4:36 PM Luke Chen <show...@gmail.com> wrote:
>
> > Hi Matthias,
> >
> > Good point! I didn't notice that. Yes, I agree it's not good to let store
> > depends on kstream package. I'll remove the stores API change later.
> (after
> > Chinese new year :) )
> >
> > Thanks for your good suggestion!
> >
> > Luke
> >
> >
> > Matthias J. Sax <mj...@apache.org> 於 2022年2月1日 週二 下午3:10 寫道:
> >
> >> Thanks for updating the KIP.
> >>
> >> It's much clearer now. I think the improvement to `Materialized` is a
> >> good one.
> >>
> >> However, I still have doubts about `Stores`. While the API change itself
> >> seems ok (even if I don't see a large benefit), it adds a dependency
> >> from the DSL classes (ie, package `kstream`) to the state store package
> >> `state`. I think it would be better to avoid "cyclic" package
> >> dependencies. While it does not matter atm, as it's all a single module,
> >> we usually try to avoid "reverse" dependencies: neither the `processor`
> >> nor `state` packages should use anything from the `kstream` package
> IMHO.
> >>
> >>
> >> -Matthias
> >>
> >>
> >>
> >> On 1/30/22 12:00 AM, Luke Chen wrote:
> >> > Hi John and Guozhang,
> >> >
> >> > Thanks for your comments.
> >> >
> >> > And @John, yes, you are right.
> >> > The goal of the improved Materialized API is to provide a way to use
> >> > im-memory store without providing the name.
> >> > So, about this comment:
> >> >
> >> >> On the other hand, I don't see how the latter of these is
> >> > more compelling than the former:
> >> > .count(Materialized.as(Stores.inMemoryKeyValueStore("count-
> >> > store")));
> >> > .count(Materialized.as(Stores.keyValueStoreSupplier(StoreTyp
> >> > e.IN_MEMORY, "count-store")));
> >> >
> >> > I think I didn't make it clear here.
> >> > The improved Materialized API is like this
> >> >
> >>
> .count(Materialized.as(Materialized.withStoreType(StoreImplType.IN_MEMORY));
> >> > // without name provided.
> >> >
> >> > I've updated the KIP to make it clear.
> >> >
> >> > Therefore, I'll keep the Materialize/Stores change and complete the
> PR.
> >> >
> >> > Thanks for your comments again!
> >> >
> >> > Luke
> >> >
> >> >
> >> >
> >> > On Sat, Jan 29, 2022 at 7:46 AM John Roesler <vvcep...@apache.org>
> >> wrote:
> >> >
> >> >> Hi Luke,
> >> >>
> >> >> Thanks for the KIP!
> >> >>
> >> >> I'm +1 (binding) on your KIP.
> >> >>
> >> >> Regarding this last question about chaning Materialized
> >> >> and/or Stores, I think it might actually be best to drop
> >> >> that part of the proposal.
> >> >>
> >> >> The primary benefit of your proposal is in the cases when
> >> >> the user doesn't want to specify the store type at all and
> >> >> just, as a blanket, use in-memory stores across the whole
> >> >> topology instead of rocksDB ones. For that, we have the
> >> >> config you proposed.
> >> >>
> >> >> As I read it, the Materialized part of the proposal was
> >> >> secondary; to allow users to override the default storage
> >> >> engine on a per-operation basis without having to bother
> >> >> about providing a full-fledged store supplier. In other
> >> >> words, today, if you want an in-memory store on a grouped
> >> >> stream, you have to do:
> >> >>
> >> >> .count(Materialized.as(Stores.inMemoryKeyValueStore("count-
> >> >> store")));
> >> >>
> >> >> What if you didn't care about the name but wanted it to be
> >> >> in memory? Well, you're out of luck.
> >> >>
> >> >> Therefore, I think there's significant value in modifying
> >> >> the DSL to allow users to orthogonally specify the storage
> >> >> engine and the name of the store, as in your KIP as written.
> >> >>
> >> >> On the other hand, I don't see how the latter of these is
> >> >> more compelling than the former:
> >> >> .count(Materialized.as(Stores.inMemoryKeyValueStore("count-
> >> >> store")));
> >> >> .count(Materialized.as(Stores.keyValueStoreSupplier(StoreTyp
> >> >> e.IN_MEMORY, "count-store")));
> >> >>
> >> >>
> >> >> Regardless, I don't want to let perfect be the enemy of
> >> >> good. Like I said, I think that the key benefit you're
> >> >> really going for is the config, so maybe you want to just
> >> >> drop the Materialize/Stores aspect and simplify the
> >> >> proposal. Or if you want to keep the latter, I'm fine with
> >> >> whatever approach you feel is best (which is why I still
> >> >> voted).
> >> >>
> >> >> This feels like the kind of thing that won't really be
> >> >> crystal clear until the PR is under review (and I'd
> >> >> encourage you and the reviewer to pay particular attention
> >> >> to how the new APIs actually look when used in the tests).
> >> >>
> >> >> Thanks again! People have been asking for this for a long
> >> >> time.
> >> >> -John
> >> >>
> >> >>
> >> >> On Fri, 2022-01-28 at 13:46 -0800, Guozhang Wang wrote:
> >> >>> Hi Luke,
> >> >>>
> >> >>> I'm in favor of using the newly proposed
> `#sessionStore(StoreType..)`
> >> and
> >> >>> deprecating the existing `#persistenSessionStore` etc.
> >> >>>
> >> >>> Thanks,
> >> >>> Guozhang
> >> >>>
> >> >>> On Tue, Jan 25, 2022 at 12:17 AM Luke Chen <show...@gmail.com>
> wrote:
> >> >>>
> >> >>>> Thanks Matthias!
> >> >>>>
> >> >>>> I agree we could deprecate the existing ones, and add the one with
> >> >>>> storeType parameter.
> >> >>>>
> >> >>>> That is:
> >> >>>> @deprecated
> >> >>>> Stores#persistentSessionStore(...)
> >> >>>> @deprecated
> >> >>>> Stores#inMemorySessionStore(...)
> >> >>>> @new added with an additional storeType parameter (IN_MEMORY or
> >> >> ROCKS_DB)
> >> >>>> Stores#sessionStoreSupplier(StoreType storeType, ...)
> >> >>>>
> >> >>>> Let's see what others think about it.
> >> >>>>
> >> >>>> Thank you.
> >> >>>> Luke
> >> >>>>
> >> >>>> On Tue, Jan 25, 2022 at 4:01 PM Matthias J. Sax <mj...@apache.org>
> >> >> wrote:
> >> >>>>
> >> >>>>> Thanks,
> >> >>>>>
> >> >>>>> There is already `Stores.persistentSessionStore` and
> >> >>>>> `Stores.inMemorySessionStore`. From a DSL code POV, I don't see
> >> large
> >> >>>>> benefits to add a new one, but it also does not hurt.
> >> >>>>>
> >> >>>>> Do you propose to add the third one only, or to also deprecate the
> >> >>>>> existing ones? In general, we should avoid to extend the API
> surface
> >> >>>>> area, so it could be a good simplification is we plan to remove
> the
> >> >>>>> existing ones?
> >> >>>>>
> >> >>>>> Btw: we could name the new method just `sessionStoreSupplier` for
> >> >>>>> simplicity (especially, if we deprecate the existing ones)?
> >> >>>>>
> >> >>>>> Not sure what others think. I am fine adding it, if we deprecate
> the
> >> >>>>> existing ones.
> >> >>>>>
> >> >>>>> -Matthias
> >> >>>>>
> >> >>>>>
> >> >>>>> On 1/24/22 5:03 PM, Luke Chen wrote:
> >> >>>>>> Hi Matthias,
> >> >>>>>>
> >> >>>>>> I didn't "save" the change. >.<
> >> >>>>>> Anyway, you can refer to this WIP PR to have better understanding
> >> >>>>> why/what
> >> >>>>>> the new API is:
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>
> >>
> https://github.com/apache/kafka/pull/11705/files#diff-c552e58e01169886c5d8b8b149f5c8cd48ea1fc1c3d7b932d055d3df9a00e1b5R464-R477
> >> >>>>>>
> >> >>>>>> It's not necessary, actually, but it can make the implementation
> >> >>>> cleaner.
> >> >>>>>> If you think this change is unnecessary and will make the
> `Stores`
> >> >> API
> >> >>>>> more
> >> >>>>>> complicated, it's fine to me.
> >> >>>>>>
> >> >>>>>> I'll update the KIP after we have a conclusion for it.
> >> >>>>>>
> >> >>>>>> Thank you.
> >> >>>>>> Luke
> >> >>>>>>
> >> >>>>>> On Tue, Jan 25, 2022 at 2:37 AM Matthias J. Sax <
> mj...@apache.org>
> >> >>>>> wrote:
> >> >>>>>>
> >> >>>>>>> I don't see the KIP update? Did you hit "save"?
> >> >>>>>>>
> >> >>>>>>> Also, the formatting in your email for the new methods is hard
> to
> >> >>>> read.
> >> >>>>>>> Not sure atm why we need the API change? Can you elaborate? what
> >> >> does
> >> >>>>>>>
> >> >>>>>>>> I found it'd be better
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> -Matthias
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> On 1/24/22 2:29 AM, Luke Chen wrote:
> >> >>>>>>>> Thanks for all your votes.
> >> >>>>>>>>
> >> >>>>>>>> During the implementation, I found it'd be better to have
> >> >> helper
> >> >>>>> methods
> >> >>>>>>> in
> >> >>>>>>>> `Stores`, to be able to get the store supplier by the store
> >> >> type:
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> *public static SessionBytesStoreSupplier
> >> >>>>>>>> sessionStoreSupplierByStoreType()public static
> >> >>>> WindowBytesStoreSupplier
> >> >>>>>>>> windowStoreSupplierByStoreType()public static
> >> >>>>> KeyValueBytesStoreSupplier
> >> >>>>>>>> keyValueStoreSupplierByStoreType()*
> >> >>>>>>>>
> >> >>>>>>>> I've also updated in the KIP.
> >> >>>>>>>> Please let me know if you other thoughts.
> >> >>>>>>>>
> >> >>>>>>>> Also, welcome to vote for this KIP.
> >> >>>>>>>>
> >> >>>>>>>> Thank you.
> >> >>>>>>>> Luke
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Fri, Jan 21, 2022 at 4:39 AM Walker Carlson
> >> >>>>>>>> <wcarl...@confluent.io.invalid> wrote:
> >> >>>>>>>>
> >> >>>>>>>>> +1 non binding
> >> >>>>>>>>>
> >> >>>>>>>>> On Thu, Jan 20, 2022 at 2:00 PM Matthias J. Sax <
> >> >> mj...@apache.org>
> >> >>>>>>> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>>> +1 (binding)
> >> >>>>>>>>>>
> >> >>>>>>>>>> On 1/20/22 10:52 AM, Guozhang Wang wrote:
> >> >>>>>>>>>>> Thanks Luke! I'm +1 on the KIP.
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> Guozhang
> >> >>>>>>>>>>>
> >> >>>>>>>>>>> On Wed, Jan 19, 2022 at 5:58 PM Luke Chen <
> >> >> show...@gmail.com>
> >> >>>>> wrote:
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>> Hi devs,
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> I'd like to start a vote for the KIP-591: Add Kafka
> >> >> Streams
> >> >>>> config
> >> >>>>> to
> >> >>>>>>>>>> set
> >> >>>>>>>>>>>> default state store. The goal is to allow users to set
> >> >> a default
> >> >>>>>>> store
> >> >>>>>>>>>> in
> >> >>>>>>>>>>>> the config, so it can apply to all the streams.
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Detailed description can be found here:
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>
> >> >>>>>
> >> >>>>
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-591%3A+Add+Kafka+Streams+config+to+set+default+state+store
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>> Thank you.
> >> >>>>>>>>>>>> Luke
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>>
> >> >>>>>
> >> >>>>
> >> >>>
> >> >>>
> >> >>
> >> >>
> >> >
> >>
> >
>


-- 
-- Guozhang

Reply via email to