Looks good to me! I have no further comments.

Thanks again for the KIP, Aishwarya!
-John

On Fri, Sep 27, 2019 at 10:11 AM aishwarya kumar <ash26...@gmail.com> wrote:
>
> Hello John,
>
> Thank you for pointing this out to me, to maintain consistency across API's
> it does make sense to allow users to define custom names for
> their processors.
>
> I've made the change in the KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL
>
> Best,
> Aishwarya
>
> On Tue, Sep 24, 2019 at 11:54 AM John Roesler <j...@confluent.io> wrote:
>
> > Hey Aishwarya,
> >
> > Thanks for the KIP! It looks good to me, although in a post-KIP-307
> > world, we also need a "Named" parameter (to give the processor node a
> > name, as opposed to the store itself).
> >
> > This would result in a total of four overloads:
> > 1. no args
> > 2. Named
> > 3. Materialized
> > 4. Materialized, Named
> >
> > I'd like to propose a re-design of the DSL in the future to clean this
> > up, but for now, this is the pattern we have to follow.
> >
> > Thoughts?
> >
> > Thanks,
> > -John
> >
> > On Tue, Sep 24, 2019 at 9:54 AM aishwarya kumar <ash26...@gmail.com>
> > wrote:
> > >
> > > Thank you for the suggestion Matthais, i've made the necessary changes in
> > > the KIP.
> > >
> > > Keeping this thread open for further input.
> > > KIP link:
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-523%3A+Add+KStream%23toTable+to+the+Streams+DSL
> > >
> > > Best,
> > > Aishwarya
> > >
> > > On Thu, Sep 19, 2019 at 10:50 AM aishwarya kumar <ash26...@gmail.com>
> > wrote:
> > >
> > > > Thanks Matthias,
> > > >
> > > > That does make sense, let me update the KIP to reflect the
> > Materialization
> > > > scenario.
> > > >
> > > > Best,
> > > > Aishwarya
> > > >
> > > > On Tue, Sep 17, 2019, 2:49 PM Matthias J. Sax <matth...@confluent.io>
> > > > wrote:
> > > >
> > > >> Aishwarya,
> > > >>
> > > >> thanks for the KIP. Overall, I think it makes sense to allow
> > converting
> > > >> a KStream into a KTable.
> > > >>
> > > >> From the KIP:
> > > >>
> > > >> > materializing these KTables should only be allowed if the overloaded
> > > >> function with Materialized is used (and if optimization is turned on
> > it may
> > > >> still be only logically materialized if the queryable name is not
> > set).
> > > >>
> > > >> Can you elaborate? I think the behavior we want should align with the
> > > >> behavior of `StreamsBuilder#table()`.
> > > >>
> > > >> From my understanding (correct me if I am wrong) it should be:
> > > >>
> > > >> (1) If optimization is turned off, the KTable will always be
> > > >> materialized, independent which method is used. The KTable will not be
> > > >> queryable though.
> > > >>
> > > >> (2) If optimization is turned on and if `toTable()` is used, the
> > KTable
> > > >> may or may not be materialized. For this case, even if the KTable is
> > > >> materialized, the store would not be queryable.
> > > >>
> > > >> (3) If `toTable(Materialized)` is use and a `storeName` or
> > > >> `StoreSupplier` is specified, the store will always be materialized
> > and
> > > >> also be queryable. Otherwise, case (1) or (2) applies.
> > > >>
> > > >>
> > > >>
> > > >> -Matthias
> > > >>
> > > >>
> > > >> On 9/17/19 6:42 AM, aishwarya kumar wrote:
> > > >> > Hi All,
> > > >> >
> > > >> > Keeping this thread alive!!
> > > >> >
> > > >> > The aim is to add two methods Kstream.toTable() &
> > > >> > Kstream.toTable(Materialized<K,V>), so users can choose to convert
> > their
> > > >> > event stream into a changelog stream at any stage.
> > > >> > wiki link :
> > > >> >
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-523:+Add+KStream%23toTable+to+the+Streams+DSL
> > > >> > jira ticket : https://issues.apache.org/jira/browse/KAFKA-7658
> > > >> >
> > > >> > Best,
> > > >> > Aishwarya
> > > >> >
> > > >> > On Fri, Sep 13, 2019 at 10:49 AM aishwarya kumar <
> > ash26...@gmail.com>
> > > >> wrote:
> > > >> >
> > > >> >> Hello,
> > > >> >>
> > > >> >> Starting this thread to discuss KIP-532:
> > > >> >> wiki link :
> > > >> >>
> > > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-523:+Add+KStream%23toTable+to+the+Streams+DSL
> > > >> >> jira ticket : https://issues.apache.org/jira/browse/KAFKA-7658
> > > >> >>
> > > >> >> There has been some discussion around the use-case of this KIP in
> > the
> > > >> Jira
> > > >> >> ticket.
> > > >> >>
> > > >> >> Regards,
> > > >> >> Aishwarya
> > > >> >>
> > > >> >
> > > >>
> > > >>
> >

Reply via email to