You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Damian Guy <da...@gmail.com> on 2017/06/21 14:37:33 UTC

[DISCUSS] Streams DSL/StateStore Refactoring

Hi,

I'd like to get a discussion going around some of the API choices we've
made in the DLS. In particular those that relate to stateful operations
(though this could expand).
As it stands we lean heavily on overloaded methods in the API, i.e, there
are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
feel it is only going to get worse as we add more optional params. In
particular we've had some requests to be able to turn caching off, or
change log configs,  on a per operator basis (note this can be done now if
you pass in a StateStoreSupplier, but this can be a bit cumbersome).

So this is a bit of an open question. How can we change the DSL overloads
so that it flows, is simple to use and understand, and is easily extended
in the future?

One option would be to use a fluent API approach for providing the optional
params, so something like this:

groupedStream.count()
   .withStoreName("name")
   .withCachingEnabled(false)
   .withLoggingEnabled(config)
   .table()



Another option would be to provide a Builder to the count method, so it
would look something like this:
groupedStream.count(new
CountBuilder("storeName").withCachingEnabled(false).build())

Another option is to say: Hey we don't need this, what are you on about!

The above has focussed on state store related overloads, but the same ideas
could  be applied to joins etc, where we presently have many join methods
and many overloads.

Anyway, i look forward to hearing your opinions.

Thanks,
Damian

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Eno,

I am less interested in the user facing interface but more in the actual 
implementation. Any hints where I can follow the discussion on this? As 
I still want to discuss upstreaming of KAFKA-3705 with someone

Best Jan


On 21.06.2017 17:24, Eno Thereska wrote:
> (cc’ing user-list too)
>
> Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC.
>
> So following your example, I guess you’d have something like:
> .join()
>     .withKeySerdes(…)
>     .withValueSerdes(…)
>     .withJoinType(“outer”)
>
> etc?
>
> I like the approach since it still remains declarative and it’d reduce the number of overloads by quite a bit.
>
> Eno
>
>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>
>> Hi,
>>
>> I'd like to get a discussion going around some of the API choices we've
>> made in the DLS. In particular those that relate to stateful operations
>> (though this could expand).
>> As it stands we lean heavily on overloaded methods in the API, i.e, there
>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
>> feel it is only going to get worse as we add more optional params. In
>> particular we've had some requests to be able to turn caching off, or
>> change log configs,  on a per operator basis (note this can be done now if
>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>
>> So this is a bit of an open question. How can we change the DSL overloads
>> so that it flows, is simple to use and understand, and is easily extended
>> in the future?
>>
>> One option would be to use a fluent API approach for providing the optional
>> params, so something like this:
>>
>> groupedStream.count()
>>    .withStoreName("name")
>>    .withCachingEnabled(false)
>>    .withLoggingEnabled(config)
>>    .table()
>>
>>
>>
>> Another option would be to provide a Builder to the count method, so it
>> would look something like this:
>> groupedStream.count(new
>> CountBuilder("storeName").withCachingEnabled(false).build())
>>
>> Another option is to say: Hey we don't need this, what are you on about!
>>
>> The above has focussed on state store related overloads, but the same ideas
>> could  be applied to joins etc, where we presently have many join methods
>> and many overloads.
>>
>> Anyway, i look forward to hearing your opinions.
>>
>> Thanks,
>> Damian


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
I am with Gouzhang here.

I think all the suggestions are far to short-sighted. Especially this 
wired materialize(String) call is broken totally and people go nuts 
about how this will look. + Implementing more and better joins, not this 
wired one we got currently. Implementing an one to many join I couln't 
get away without 3 highly complex value mappers

                                        final ValueMapper<VR, K> 
keyExtractor,
                                        final ValueMapper<KL, K> 
joinPrefixFaker,
                                        final ValueMapper<K, KL> 
leftKeyExtractor,

in addition to the one joiner of course

                                        final ValueJoiner<VL, VR, V> joiner,

how to specify if its outer or inner is for sure the smallest problem we 
are going to face with proper join semantics. What the resulting Key 
will be is is also highly discussable. What happens to the key is very 
complex and the API has to tell the user.

Bringing this discussion into a good direction, we would need sample 
interfaces we could mock against ( as gouzhang suggested) + We need to 
know how the implementation (of joins especially) will be later. As I 
strongly recommend stopping the usage of ChangeSerde and have "properly" 
repartitioned topic. That is just sane IMO

Best Jan




On 22.06.2017 11:54, Eno Thereska wrote:
> Note that while I agree with the initial proposal (withKeySerdes, withJoinType, etc), I don't agree with things like .materialize(), .enableCaching(), .enableLogging().
>
> The former maintain the declarative DSL, while the later break the declarative part by mixing system decisions in the DSL.  I think there is a difference between the two proposals.
>
> Eno
>
>> On 22 Jun 2017, at 03:46, Guozhang Wang <wa...@gmail.com> wrote:
>>
>> I have been thinking about reducing all these overloaded functions for
>> stateful operations (there are some other places that introduces overloaded
>> functions but let's focus on these only in this discussion), what I used to
>> have is to use some "materialize" function on the KTables, like:
>>
>> ---------------------------------------
>>
>> // specifying the topology
>>
>> KStream stream1 = builder.stream();
>> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
>> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
>> supplier here any more
>>
>> // additional specs along with the topology above
>>
>> table1.materialize("queryableStoreName"); // or..
>> table1.materialize("queryableStoreName").enableCaching().enableLogging();
>> // or..
>> table1.materialize(stateStoreSupplier); // add the metrics / logging /
>> caching / windowing functionalities on top of the store, or..
>> table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
>> etc..
>>
>> ---------------------------------------
>>
>> But thinking about it more, I feel Damian's first proposal is better since
>> my proposal would likely to break the concatenation (e.g. we may not be
>> able to do sth. like "table1.filter().map().groupBy().aggregate()" if we
>> want to use different specs for the intermediate filtered KTable).
>>
>>
>> But since this is a incompatibility change, and we are going to remove the
>> compatibility annotations soon it means we only have one chance and we
>> really have to make it right. So I'd call out for anyone try to rewrite
>> your examples / demo code with the proposed new API and see if it feel
>> natural, for example, if I want to use a different storage engine than the
>> default rockDB engine how could I easily specify that with the proposed
>> APIs?
>>
>> Meanwhile Damian could you provide a formal set of APIs for people to
>> exercise on them? Also could you briefly describe how custom storage
>> engines could be swapped in with the above APIs?
>>
>>
>>
>> Guozhang
>>
>>
>> On Wed, Jun 21, 2017 at 9:08 AM, Eno Thereska <en...@gmail.com>
>> wrote:
>>
>>> To make it clear, it’s outlined by Damian, I just copy pasted what he told
>>> me in person :)
>>>
>>> Eno
>>>
>>>> On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
>>>>
>>>> +1 for the approach outlined above by Eno.
>>>>
>>>> On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com>
>>> wrote:
>>>>> Thanks Eno.
>>>>>
>>>>> Yes i agree. We could apply this same approach to most of the operations
>>>>> where we have multiple overloads, i.e., we have a single method for each
>>>>> operation that takes the required parameters and everything else is
>>>>> specified as you have done above.
>>>>>
>>>>> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com>
>>> wrote:
>>>>>> (cc’ing user-list too)
>>>>>>
>>>>>> Given that we already have StateStoreSuppliers that are configurable
>>>>> using
>>>>>> the fluent-like API, probably it’s worth discussing the other examples
>>>>> with
>>>>>> joins and serdes first since those have many overloads and are in need
>>> of
>>>>>> some TLC.
>>>>>>
>>>>>> So following your example, I guess you’d have something like:
>>>>>> .join()
>>>>>>   .withKeySerdes(…)
>>>>>>   .withValueSerdes(…)
>>>>>>   .withJoinType(“outer”)
>>>>>>
>>>>>> etc?
>>>>>>
>>>>>> I like the approach since it still remains declarative and it’d reduce
>>>>> the
>>>>>> number of overloads by quite a bit.
>>>>>>
>>>>>> Eno
>>>>>>
>>>>>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> I'd like to get a discussion going around some of the API choices
>>> we've
>>>>>>> made in the DLS. In particular those that relate to stateful
>>> operations
>>>>>>> (though this could expand).
>>>>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>>>>> there
>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>>>>> i
>>>>>>> feel it is only going to get worse as we add more optional params. In
>>>>>>> particular we've had some requests to be able to turn caching off, or
>>>>>>> change log configs,  on a per operator basis (note this can be done
>>> now
>>>>>> if
>>>>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>>>>>
>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>> overloads
>>>>>>> so that it flows, is simple to use and understand, and is easily
>>>>> extended
>>>>>>> in the future?
>>>>>>>
>>>>>>> One option would be to use a fluent API approach for providing the
>>>>>> optional
>>>>>>> params, so something like this:
>>>>>>>
>>>>>>> groupedStream.count()
>>>>>>> .withStoreName("name")
>>>>>>> .withCachingEnabled(false)
>>>>>>> .withLoggingEnabled(config)
>>>>>>> .table()
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Another option would be to provide a Builder to the count method, so
>>> it
>>>>>>> would look something like this:
>>>>>>> groupedStream.count(new
>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>
>>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>>> about!
>>>>>>> The above has focussed on state store related overloads, but the same
>>>>>> ideas
>>>>>>> could  be applied to joins etc, where we presently have many join
>>>>> methods
>>>>>>> and many overloads.
>>>>>>>
>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Damian
>>>>>>
>>>
>>
>> -- 
>> -- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
Note that while I agree with the initial proposal (withKeySerdes, withJoinType, etc), I don't agree with things like .materialize(), .enableCaching(), .enableLogging(). 

The former maintain the declarative DSL, while the later break the declarative part by mixing system decisions in the DSL.  I think there is a difference between the two proposals.

Eno

> On 22 Jun 2017, at 03:46, Guozhang Wang <wa...@gmail.com> wrote:
> 
> I have been thinking about reducing all these overloaded functions for
> stateful operations (there are some other places that introduces overloaded
> functions but let's focus on these only in this discussion), what I used to
> have is to use some "materialize" function on the KTables, like:
> 
> ---------------------------------------
> 
> // specifying the topology
> 
> KStream stream1 = builder.stream();
> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
> supplier here any more
> 
> // additional specs along with the topology above
> 
> table1.materialize("queryableStoreName"); // or..
> table1.materialize("queryableStoreName").enableCaching().enableLogging();
> // or..
> table1.materialize(stateStoreSupplier); // add the metrics / logging /
> caching / windowing functionalities on top of the store, or..
> table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
> etc..
> 
> ---------------------------------------
> 
> But thinking about it more, I feel Damian's first proposal is better since
> my proposal would likely to break the concatenation (e.g. we may not be
> able to do sth. like "table1.filter().map().groupBy().aggregate()" if we
> want to use different specs for the intermediate filtered KTable).
> 
> 
> But since this is a incompatibility change, and we are going to remove the
> compatibility annotations soon it means we only have one chance and we
> really have to make it right. So I'd call out for anyone try to rewrite
> your examples / demo code with the proposed new API and see if it feel
> natural, for example, if I want to use a different storage engine than the
> default rockDB engine how could I easily specify that with the proposed
> APIs?
> 
> Meanwhile Damian could you provide a formal set of APIs for people to
> exercise on them? Also could you briefly describe how custom storage
> engines could be swapped in with the above APIs?
> 
> 
> 
> Guozhang
> 
> 
> On Wed, Jun 21, 2017 at 9:08 AM, Eno Thereska <en...@gmail.com>
> wrote:
> 
>> To make it clear, it’s outlined by Damian, I just copy pasted what he told
>> me in person :)
>> 
>> Eno
>> 
>>> On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
>>> 
>>> +1 for the approach outlined above by Eno.
>>> 
>>> On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com>
>> wrote:
>>> 
>>>> Thanks Eno.
>>>> 
>>>> Yes i agree. We could apply this same approach to most of the operations
>>>> where we have multiple overloads, i.e., we have a single method for each
>>>> operation that takes the required parameters and everything else is
>>>> specified as you have done above.
>>>> 
>>>> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com>
>> wrote:
>>>> 
>>>>> (cc’ing user-list too)
>>>>> 
>>>>> Given that we already have StateStoreSuppliers that are configurable
>>>> using
>>>>> the fluent-like API, probably it’s worth discussing the other examples
>>>> with
>>>>> joins and serdes first since those have many overloads and are in need
>> of
>>>>> some TLC.
>>>>> 
>>>>> So following your example, I guess you’d have something like:
>>>>> .join()
>>>>>  .withKeySerdes(…)
>>>>>  .withValueSerdes(…)
>>>>>  .withJoinType(“outer”)
>>>>> 
>>>>> etc?
>>>>> 
>>>>> I like the approach since it still remains declarative and it’d reduce
>>>> the
>>>>> number of overloads by quite a bit.
>>>>> 
>>>>> Eno
>>>>> 
>>>>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> I'd like to get a discussion going around some of the API choices
>> we've
>>>>>> made in the DLS. In particular those that relate to stateful
>> operations
>>>>>> (though this could expand).
>>>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>>>> there
>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>>>> i
>>>>>> feel it is only going to get worse as we add more optional params. In
>>>>>> particular we've had some requests to be able to turn caching off, or
>>>>>> change log configs,  on a per operator basis (note this can be done
>> now
>>>>> if
>>>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>>>> 
>>>>>> So this is a bit of an open question. How can we change the DSL
>>>> overloads
>>>>>> so that it flows, is simple to use and understand, and is easily
>>>> extended
>>>>>> in the future?
>>>>>> 
>>>>>> One option would be to use a fluent API approach for providing the
>>>>> optional
>>>>>> params, so something like this:
>>>>>> 
>>>>>> groupedStream.count()
>>>>>> .withStoreName("name")
>>>>>> .withCachingEnabled(false)
>>>>>> .withLoggingEnabled(config)
>>>>>> .table()
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Another option would be to provide a Builder to the count method, so
>> it
>>>>>> would look something like this:
>>>>>> groupedStream.count(new
>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>> 
>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>> about!
>>>>>> 
>>>>>> The above has focussed on state store related overloads, but the same
>>>>> ideas
>>>>>> could  be applied to joins etc, where we presently have many join
>>>> methods
>>>>>> and many overloads.
>>>>>> 
>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>> 
>>>>>> Thanks,
>>>>>> Damian
>>>>> 
>>>>> 
>>>> 
>> 
>> 
> 
> 
> -- 
> -- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
Note that while I agree with the initial proposal (withKeySerdes, withJoinType, etc), I don't agree with things like .materialize(), .enableCaching(), .enableLogging(). 

The former maintain the declarative DSL, while the later break the declarative part by mixing system decisions in the DSL.  I think there is a difference between the two proposals.

Eno

> On 22 Jun 2017, at 03:46, Guozhang Wang <wa...@gmail.com> wrote:
> 
> I have been thinking about reducing all these overloaded functions for
> stateful operations (there are some other places that introduces overloaded
> functions but let's focus on these only in this discussion), what I used to
> have is to use some "materialize" function on the KTables, like:
> 
> ---------------------------------------
> 
> // specifying the topology
> 
> KStream stream1 = builder.stream();
> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
> supplier here any more
> 
> // additional specs along with the topology above
> 
> table1.materialize("queryableStoreName"); // or..
> table1.materialize("queryableStoreName").enableCaching().enableLogging();
> // or..
> table1.materialize(stateStoreSupplier); // add the metrics / logging /
> caching / windowing functionalities on top of the store, or..
> table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
> etc..
> 
> ---------------------------------------
> 
> But thinking about it more, I feel Damian's first proposal is better since
> my proposal would likely to break the concatenation (e.g. we may not be
> able to do sth. like "table1.filter().map().groupBy().aggregate()" if we
> want to use different specs for the intermediate filtered KTable).
> 
> 
> But since this is a incompatibility change, and we are going to remove the
> compatibility annotations soon it means we only have one chance and we
> really have to make it right. So I'd call out for anyone try to rewrite
> your examples / demo code with the proposed new API and see if it feel
> natural, for example, if I want to use a different storage engine than the
> default rockDB engine how could I easily specify that with the proposed
> APIs?
> 
> Meanwhile Damian could you provide a formal set of APIs for people to
> exercise on them? Also could you briefly describe how custom storage
> engines could be swapped in with the above APIs?
> 
> 
> 
> Guozhang
> 
> 
> On Wed, Jun 21, 2017 at 9:08 AM, Eno Thereska <en...@gmail.com>
> wrote:
> 
>> To make it clear, it’s outlined by Damian, I just copy pasted what he told
>> me in person :)
>> 
>> Eno
>> 
>>> On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
>>> 
>>> +1 for the approach outlined above by Eno.
>>> 
>>> On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com>
>> wrote:
>>> 
>>>> Thanks Eno.
>>>> 
>>>> Yes i agree. We could apply this same approach to most of the operations
>>>> where we have multiple overloads, i.e., we have a single method for each
>>>> operation that takes the required parameters and everything else is
>>>> specified as you have done above.
>>>> 
>>>> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com>
>> wrote:
>>>> 
>>>>> (cc’ing user-list too)
>>>>> 
>>>>> Given that we already have StateStoreSuppliers that are configurable
>>>> using
>>>>> the fluent-like API, probably it’s worth discussing the other examples
>>>> with
>>>>> joins and serdes first since those have many overloads and are in need
>> of
>>>>> some TLC.
>>>>> 
>>>>> So following your example, I guess you’d have something like:
>>>>> .join()
>>>>>  .withKeySerdes(…)
>>>>>  .withValueSerdes(…)
>>>>>  .withJoinType(“outer”)
>>>>> 
>>>>> etc?
>>>>> 
>>>>> I like the approach since it still remains declarative and it’d reduce
>>>> the
>>>>> number of overloads by quite a bit.
>>>>> 
>>>>> Eno
>>>>> 
>>>>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> I'd like to get a discussion going around some of the API choices
>> we've
>>>>>> made in the DLS. In particular those that relate to stateful
>> operations
>>>>>> (though this could expand).
>>>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>>>> there
>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>>>> i
>>>>>> feel it is only going to get worse as we add more optional params. In
>>>>>> particular we've had some requests to be able to turn caching off, or
>>>>>> change log configs,  on a per operator basis (note this can be done
>> now
>>>>> if
>>>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>>>> 
>>>>>> So this is a bit of an open question. How can we change the DSL
>>>> overloads
>>>>>> so that it flows, is simple to use and understand, and is easily
>>>> extended
>>>>>> in the future?
>>>>>> 
>>>>>> One option would be to use a fluent API approach for providing the
>>>>> optional
>>>>>> params, so something like this:
>>>>>> 
>>>>>> groupedStream.count()
>>>>>> .withStoreName("name")
>>>>>> .withCachingEnabled(false)
>>>>>> .withLoggingEnabled(config)
>>>>>> .table()
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> Another option would be to provide a Builder to the count method, so
>> it
>>>>>> would look something like this:
>>>>>> groupedStream.count(new
>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>> 
>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>> about!
>>>>>> 
>>>>>> The above has focussed on state store related overloads, but the same
>>>>> ideas
>>>>>> could  be applied to joins etc, where we presently have many join
>>>> methods
>>>>>> and many overloads.
>>>>>> 
>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>> 
>>>>>> Thanks,
>>>>>> Damian
>>>>> 
>>>>> 
>>>> 
>> 
>> 
> 
> 
> -- 
> -- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
I have been thinking about reducing all these overloaded functions for
stateful operations (there are some other places that introduces overloaded
functions but let's focus on these only in this discussion), what I used to
have is to use some "materialize" function on the KTables, like:

---------------------------------------

// specifying the topology

KStream stream1 = builder.stream();
KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
sessionMerger, sessionWindows);  // do not allow to pass-in a state store
supplier here any more

// additional specs along with the topology above

table1.materialize("queryableStoreName"); // or..
table1.materialize("queryableStoreName").enableCaching().enableLogging();
// or..
table1.materialize(stateStoreSupplier); // add the metrics / logging /
caching / windowing functionalities on top of the store, or..
table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
etc..

---------------------------------------

But thinking about it more, I feel Damian's first proposal is better since
my proposal would likely to break the concatenation (e.g. we may not be
able to do sth. like "table1.filter().map().groupBy().aggregate()" if we
want to use different specs for the intermediate filtered KTable).


But since this is a incompatibility change, and we are going to remove the
compatibility annotations soon it means we only have one chance and we
really have to make it right. So I'd call out for anyone try to rewrite
your examples / demo code with the proposed new API and see if it feel
natural, for example, if I want to use a different storage engine than the
default rockDB engine how could I easily specify that with the proposed
APIs?

Meanwhile Damian could you provide a formal set of APIs for people to
exercise on them? Also could you briefly describe how custom storage
engines could be swapped in with the above APIs?



Guozhang


On Wed, Jun 21, 2017 at 9:08 AM, Eno Thereska <en...@gmail.com>
wrote:

> To make it clear, it’s outlined by Damian, I just copy pasted what he told
> me in person :)
>
> Eno
>
> > On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
> >
> > +1 for the approach outlined above by Eno.
> >
> > On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com>
> wrote:
> >
> >> Thanks Eno.
> >>
> >> Yes i agree. We could apply this same approach to most of the operations
> >> where we have multiple overloads, i.e., we have a single method for each
> >> operation that takes the required parameters and everything else is
> >> specified as you have done above.
> >>
> >> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com>
> wrote:
> >>
> >>> (cc’ing user-list too)
> >>>
> >>> Given that we already have StateStoreSuppliers that are configurable
> >> using
> >>> the fluent-like API, probably it’s worth discussing the other examples
> >> with
> >>> joins and serdes first since those have many overloads and are in need
> of
> >>> some TLC.
> >>>
> >>> So following your example, I guess you’d have something like:
> >>> .join()
> >>>   .withKeySerdes(…)
> >>>   .withValueSerdes(…)
> >>>   .withJoinType(“outer”)
> >>>
> >>> etc?
> >>>
> >>> I like the approach since it still remains declarative and it’d reduce
> >> the
> >>> number of overloads by quite a bit.
> >>>
> >>> Eno
> >>>
> >>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> >>>>
> >>>> Hi,
> >>>>
> >>>> I'd like to get a discussion going around some of the API choices
> we've
> >>>> made in the DLS. In particular those that relate to stateful
> operations
> >>>> (though this could expand).
> >>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> >> there
> >>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
> >> i
> >>>> feel it is only going to get worse as we add more optional params. In
> >>>> particular we've had some requests to be able to turn caching off, or
> >>>> change log configs,  on a per operator basis (note this can be done
> now
> >>> if
> >>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>>>
> >>>> So this is a bit of an open question. How can we change the DSL
> >> overloads
> >>>> so that it flows, is simple to use and understand, and is easily
> >> extended
> >>>> in the future?
> >>>>
> >>>> One option would be to use a fluent API approach for providing the
> >>> optional
> >>>> params, so something like this:
> >>>>
> >>>> groupedStream.count()
> >>>>  .withStoreName("name")
> >>>>  .withCachingEnabled(false)
> >>>>  .withLoggingEnabled(config)
> >>>>  .table()
> >>>>
> >>>>
> >>>>
> >>>> Another option would be to provide a Builder to the count method, so
> it
> >>>> would look something like this:
> >>>> groupedStream.count(new
> >>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>
> >>>> Another option is to say: Hey we don't need this, what are you on
> >> about!
> >>>>
> >>>> The above has focussed on state store related overloads, but the same
> >>> ideas
> >>>> could  be applied to joins etc, where we presently have many join
> >> methods
> >>>> and many overloads.
> >>>>
> >>>> Anyway, i look forward to hearing your opinions.
> >>>>
> >>>> Thanks,
> >>>> Damian
> >>>
> >>>
> >>
>
>


-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
To make it clear, it’s outlined by Damian, I just copy pasted what he told me in person :)

Eno

> On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
> 
> +1 for the approach outlined above by Eno.
> 
> On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com> wrote:
> 
>> Thanks Eno.
>> 
>> Yes i agree. We could apply this same approach to most of the operations
>> where we have multiple overloads, i.e., we have a single method for each
>> operation that takes the required parameters and everything else is
>> specified as you have done above.
>> 
>> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:
>> 
>>> (cc’ing user-list too)
>>> 
>>> Given that we already have StateStoreSuppliers that are configurable
>> using
>>> the fluent-like API, probably it’s worth discussing the other examples
>> with
>>> joins and serdes first since those have many overloads and are in need of
>>> some TLC.
>>> 
>>> So following your example, I guess you’d have something like:
>>> .join()
>>>   .withKeySerdes(…)
>>>   .withValueSerdes(…)
>>>   .withJoinType(“outer”)
>>> 
>>> etc?
>>> 
>>> I like the approach since it still remains declarative and it’d reduce
>> the
>>> number of overloads by quite a bit.
>>> 
>>> Eno
>>> 
>>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>>> 
>>>> Hi,
>>>> 
>>>> I'd like to get a discussion going around some of the API choices we've
>>>> made in the DLS. In particular those that relate to stateful operations
>>>> (though this could expand).
>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>> there
>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>> i
>>>> feel it is only going to get worse as we add more optional params. In
>>>> particular we've had some requests to be able to turn caching off, or
>>>> change log configs,  on a per operator basis (note this can be done now
>>> if
>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>> 
>>>> So this is a bit of an open question. How can we change the DSL
>> overloads
>>>> so that it flows, is simple to use and understand, and is easily
>> extended
>>>> in the future?
>>>> 
>>>> One option would be to use a fluent API approach for providing the
>>> optional
>>>> params, so something like this:
>>>> 
>>>> groupedStream.count()
>>>>  .withStoreName("name")
>>>>  .withCachingEnabled(false)
>>>>  .withLoggingEnabled(config)
>>>>  .table()
>>>> 
>>>> 
>>>> 
>>>> Another option would be to provide a Builder to the count method, so it
>>>> would look something like this:
>>>> groupedStream.count(new
>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>> 
>>>> Another option is to say: Hey we don't need this, what are you on
>> about!
>>>> 
>>>> The above has focussed on state store related overloads, but the same
>>> ideas
>>>> could  be applied to joins etc, where we presently have many join
>> methods
>>>> and many overloads.
>>>> 
>>>> Anyway, i look forward to hearing your opinions.
>>>> 
>>>> Thanks,
>>>> Damian
>>> 
>>> 
>> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
To make it clear, it’s outlined by Damian, I just copy pasted what he told me in person :)

Eno

> On Jun 21, 2017, at 4:40 PM, Bill Bejeck <bb...@gmail.com> wrote:
> 
> +1 for the approach outlined above by Eno.
> 
> On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com> wrote:
> 
>> Thanks Eno.
>> 
>> Yes i agree. We could apply this same approach to most of the operations
>> where we have multiple overloads, i.e., we have a single method for each
>> operation that takes the required parameters and everything else is
>> specified as you have done above.
>> 
>> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:
>> 
>>> (cc’ing user-list too)
>>> 
>>> Given that we already have StateStoreSuppliers that are configurable
>> using
>>> the fluent-like API, probably it’s worth discussing the other examples
>> with
>>> joins and serdes first since those have many overloads and are in need of
>>> some TLC.
>>> 
>>> So following your example, I guess you’d have something like:
>>> .join()
>>>   .withKeySerdes(…)
>>>   .withValueSerdes(…)
>>>   .withJoinType(“outer”)
>>> 
>>> etc?
>>> 
>>> I like the approach since it still remains declarative and it’d reduce
>> the
>>> number of overloads by quite a bit.
>>> 
>>> Eno
>>> 
>>>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>>> 
>>>> Hi,
>>>> 
>>>> I'd like to get a discussion going around some of the API choices we've
>>>> made in the DLS. In particular those that relate to stateful operations
>>>> (though this could expand).
>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>> there
>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>> i
>>>> feel it is only going to get worse as we add more optional params. In
>>>> particular we've had some requests to be able to turn caching off, or
>>>> change log configs,  on a per operator basis (note this can be done now
>>> if
>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>> 
>>>> So this is a bit of an open question. How can we change the DSL
>> overloads
>>>> so that it flows, is simple to use and understand, and is easily
>> extended
>>>> in the future?
>>>> 
>>>> One option would be to use a fluent API approach for providing the
>>> optional
>>>> params, so something like this:
>>>> 
>>>> groupedStream.count()
>>>>  .withStoreName("name")
>>>>  .withCachingEnabled(false)
>>>>  .withLoggingEnabled(config)
>>>>  .table()
>>>> 
>>>> 
>>>> 
>>>> Another option would be to provide a Builder to the count method, so it
>>>> would look something like this:
>>>> groupedStream.count(new
>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>> 
>>>> Another option is to say: Hey we don't need this, what are you on
>> about!
>>>> 
>>>> The above has focussed on state store related overloads, but the same
>>> ideas
>>>> could  be applied to joins etc, where we presently have many join
>> methods
>>>> and many overloads.
>>>> 
>>>> Anyway, i look forward to hearing your opinions.
>>>> 
>>>> Thanks,
>>>> Damian
>>> 
>>> 
>> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Bill Bejeck <bb...@gmail.com>.
+1 for the approach outlined above by Eno.

On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com> wrote:

> Thanks Eno.
>
> Yes i agree. We could apply this same approach to most of the operations
> where we have multiple overloads, i.e., we have a single method for each
> operation that takes the required parameters and everything else is
> specified as you have done above.
>
> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:
>
> > (cc’ing user-list too)
> >
> > Given that we already have StateStoreSuppliers that are configurable
> using
> > the fluent-like API, probably it’s worth discussing the other examples
> with
> > joins and serdes first since those have many overloads and are in need of
> > some TLC.
> >
> > So following your example, I guess you’d have something like:
> > .join()
> >    .withKeySerdes(…)
> >    .withValueSerdes(…)
> >    .withJoinType(“outer”)
> >
> > etc?
> >
> > I like the approach since it still remains declarative and it’d reduce
> the
> > number of overloads by quite a bit.
> >
> > Eno
> >
> > > On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> > >
> > > Hi,
> > >
> > > I'd like to get a discussion going around some of the API choices we've
> > > made in the DLS. In particular those that relate to stateful operations
> > > (though this could expand).
> > > As it stands we lean heavily on overloaded methods in the API, i.e,
> there
> > > are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
> i
> > > feel it is only going to get worse as we add more optional params. In
> > > particular we've had some requests to be able to turn caching off, or
> > > change log configs,  on a per operator basis (note this can be done now
> > if
> > > you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> > >
> > > So this is a bit of an open question. How can we change the DSL
> overloads
> > > so that it flows, is simple to use and understand, and is easily
> extended
> > > in the future?
> > >
> > > One option would be to use a fluent API approach for providing the
> > optional
> > > params, so something like this:
> > >
> > > groupedStream.count()
> > >   .withStoreName("name")
> > >   .withCachingEnabled(false)
> > >   .withLoggingEnabled(config)
> > >   .table()
> > >
> > >
> > >
> > > Another option would be to provide a Builder to the count method, so it
> > > would look something like this:
> > > groupedStream.count(new
> > > CountBuilder("storeName").withCachingEnabled(false).build())
> > >
> > > Another option is to say: Hey we don't need this, what are you on
> about!
> > >
> > > The above has focussed on state store related overloads, but the same
> > ideas
> > > could  be applied to joins etc, where we presently have many join
> methods
> > > and many overloads.
> > >
> > > Anyway, i look forward to hearing your opinions.
> > >
> > > Thanks,
> > > Damian
> >
> >
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Bill Bejeck <bb...@gmail.com>.
+1 for the approach outlined above by Eno.

On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy <da...@gmail.com> wrote:

> Thanks Eno.
>
> Yes i agree. We could apply this same approach to most of the operations
> where we have multiple overloads, i.e., we have a single method for each
> operation that takes the required parameters and everything else is
> specified as you have done above.
>
> On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:
>
> > (cc’ing user-list too)
> >
> > Given that we already have StateStoreSuppliers that are configurable
> using
> > the fluent-like API, probably it’s worth discussing the other examples
> with
> > joins and serdes first since those have many overloads and are in need of
> > some TLC.
> >
> > So following your example, I guess you’d have something like:
> > .join()
> >    .withKeySerdes(…)
> >    .withValueSerdes(…)
> >    .withJoinType(“outer”)
> >
> > etc?
> >
> > I like the approach since it still remains declarative and it’d reduce
> the
> > number of overloads by quite a bit.
> >
> > Eno
> >
> > > On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> > >
> > > Hi,
> > >
> > > I'd like to get a discussion going around some of the API choices we've
> > > made in the DLS. In particular those that relate to stateful operations
> > > (though this could expand).
> > > As it stands we lean heavily on overloaded methods in the API, i.e,
> there
> > > are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
> i
> > > feel it is only going to get worse as we add more optional params. In
> > > particular we've had some requests to be able to turn caching off, or
> > > change log configs,  on a per operator basis (note this can be done now
> > if
> > > you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> > >
> > > So this is a bit of an open question. How can we change the DSL
> overloads
> > > so that it flows, is simple to use and understand, and is easily
> extended
> > > in the future?
> > >
> > > One option would be to use a fluent API approach for providing the
> > optional
> > > params, so something like this:
> > >
> > > groupedStream.count()
> > >   .withStoreName("name")
> > >   .withCachingEnabled(false)
> > >   .withLoggingEnabled(config)
> > >   .table()
> > >
> > >
> > >
> > > Another option would be to provide a Builder to the count method, so it
> > > would look something like this:
> > > groupedStream.count(new
> > > CountBuilder("storeName").withCachingEnabled(false).build())
> > >
> > > Another option is to say: Hey we don't need this, what are you on
> about!
> > >
> > > The above has focussed on state store related overloads, but the same
> > ideas
> > > could  be applied to joins etc, where we presently have many join
> methods
> > > and many overloads.
> > >
> > > Anyway, i look forward to hearing your opinions.
> > >
> > > Thanks,
> > > Damian
> >
> >
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks Eno.

Yes i agree. We could apply this same approach to most of the operations
where we have multiple overloads, i.e., we have a single method for each
operation that takes the required parameters and everything else is
specified as you have done above.

On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:

> (cc’ing user-list too)
>
> Given that we already have StateStoreSuppliers that are configurable using
> the fluent-like API, probably it’s worth discussing the other examples with
> joins and serdes first since those have many overloads and are in need of
> some TLC.
>
> So following your example, I guess you’d have something like:
> .join()
>    .withKeySerdes(…)
>    .withValueSerdes(…)
>    .withJoinType(“outer”)
>
> etc?
>
> I like the approach since it still remains declarative and it’d reduce the
> number of overloads by quite a bit.
>
> Eno
>
> > On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> >
> > Hi,
> >
> > I'd like to get a discussion going around some of the API choices we've
> > made in the DLS. In particular those that relate to stateful operations
> > (though this could expand).
> > As it stands we lean heavily on overloaded methods in the API, i.e, there
> > are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> > feel it is only going to get worse as we add more optional params. In
> > particular we've had some requests to be able to turn caching off, or
> > change log configs,  on a per operator basis (note this can be done now
> if
> > you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >
> > So this is a bit of an open question. How can we change the DSL overloads
> > so that it flows, is simple to use and understand, and is easily extended
> > in the future?
> >
> > One option would be to use a fluent API approach for providing the
> optional
> > params, so something like this:
> >
> > groupedStream.count()
> >   .withStoreName("name")
> >   .withCachingEnabled(false)
> >   .withLoggingEnabled(config)
> >   .table()
> >
> >
> >
> > Another option would be to provide a Builder to the count method, so it
> > would look something like this:
> > groupedStream.count(new
> > CountBuilder("storeName").withCachingEnabled(false).build())
> >
> > Another option is to say: Hey we don't need this, what are you on about!
> >
> > The above has focussed on state store related overloads, but the same
> ideas
> > could  be applied to joins etc, where we presently have many join methods
> > and many overloads.
> >
> > Anyway, i look forward to hearing your opinions.
> >
> > Thanks,
> > Damian
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks Eno.

Yes i agree. We could apply this same approach to most of the operations
where we have multiple overloads, i.e., we have a single method for each
operation that takes the required parameters and everything else is
specified as you have done above.

On Wed, 21 Jun 2017 at 16:24 Eno Thereska <en...@gmail.com> wrote:

> (cc’ing user-list too)
>
> Given that we already have StateStoreSuppliers that are configurable using
> the fluent-like API, probably it’s worth discussing the other examples with
> joins and serdes first since those have many overloads and are in need of
> some TLC.
>
> So following your example, I guess you’d have something like:
> .join()
>    .withKeySerdes(…)
>    .withValueSerdes(…)
>    .withJoinType(“outer”)
>
> etc?
>
> I like the approach since it still remains declarative and it’d reduce the
> number of overloads by quite a bit.
>
> Eno
>
> > On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> >
> > Hi,
> >
> > I'd like to get a discussion going around some of the API choices we've
> > made in the DLS. In particular those that relate to stateful operations
> > (though this could expand).
> > As it stands we lean heavily on overloaded methods in the API, i.e, there
> > are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> > feel it is only going to get worse as we add more optional params. In
> > particular we've had some requests to be able to turn caching off, or
> > change log configs,  on a per operator basis (note this can be done now
> if
> > you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >
> > So this is a bit of an open question. How can we change the DSL overloads
> > so that it flows, is simple to use and understand, and is easily extended
> > in the future?
> >
> > One option would be to use a fluent API approach for providing the
> optional
> > params, so something like this:
> >
> > groupedStream.count()
> >   .withStoreName("name")
> >   .withCachingEnabled(false)
> >   .withLoggingEnabled(config)
> >   .table()
> >
> >
> >
> > Another option would be to provide a Builder to the count method, so it
> > would look something like this:
> > groupedStream.count(new
> > CountBuilder("storeName").withCachingEnabled(false).build())
> >
> > Another option is to say: Hey we don't need this, what are you on about!
> >
> > The above has focussed on state store related overloads, but the same
> ideas
> > could  be applied to joins etc, where we presently have many join methods
> > and many overloads.
> >
> > Anyway, i look forward to hearing your opinions.
> >
> > Thanks,
> > Damian
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Eno,

I am less interested in the user facing interface but more in the actual 
implementation. Any hints where I can follow the discussion on this? As 
I still want to discuss upstreaming of KAFKA-3705 with someone

Best Jan


On 21.06.2017 17:24, Eno Thereska wrote:
> (cc’ing user-list too)
>
> Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC.
>
> So following your example, I guess you’d have something like:
> .join()
>     .withKeySerdes(…)
>     .withValueSerdes(…)
>     .withJoinType(“outer”)
>
> etc?
>
> I like the approach since it still remains declarative and it’d reduce the number of overloads by quite a bit.
>
> Eno
>
>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>
>> Hi,
>>
>> I'd like to get a discussion going around some of the API choices we've
>> made in the DLS. In particular those that relate to stateful operations
>> (though this could expand).
>> As it stands we lean heavily on overloaded methods in the API, i.e, there
>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
>> feel it is only going to get worse as we add more optional params. In
>> particular we've had some requests to be able to turn caching off, or
>> change log configs,  on a per operator basis (note this can be done now if
>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>
>> So this is a bit of an open question. How can we change the DSL overloads
>> so that it flows, is simple to use and understand, and is easily extended
>> in the future?
>>
>> One option would be to use a fluent API approach for providing the optional
>> params, so something like this:
>>
>> groupedStream.count()
>>    .withStoreName("name")
>>    .withCachingEnabled(false)
>>    .withLoggingEnabled(config)
>>    .table()
>>
>>
>>
>> Another option would be to provide a Builder to the count method, so it
>> would look something like this:
>> groupedStream.count(new
>> CountBuilder("storeName").withCachingEnabled(false).build())
>>
>> Another option is to say: Hey we don't need this, what are you on about!
>>
>> The above has focussed on state store related overloads, but the same ideas
>> could  be applied to joins etc, where we presently have many join methods
>> and many overloads.
>>
>> Anyway, i look forward to hearing your opinions.
>>
>> Thanks,
>> Damian


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Inline

rather sparse for the lack of time.

Sadly I can't agree to any of your arguments and I _hate_ how its gonna 
look,
but we can't have this discussion for ever.

I think I explained everything in enough detail so my points can make sense.
if someone is interested and has specific questions, can always approach me.

Otherwise I am just going to drink the kool-aid now. :(

Best Jan

On 08.08.2017 20:37, Guozhang Wang wrote:
> Hello Jan,
>
> Thanks for your feedback. Trying to explain them a bit more here since I
> think there are still a bit mis-communication here:
>
> Here are a few things I need to clarify for KIP-182 first:
>
> 1. KIP-182 is mainly about refactoring the public APIs, NOT for making any
> optimizations on the internal implementations. So we only care that these
> public APIs changes do not forbid us to make the internal implementations
> in the near future.
>
> To give you a concrete example, as you mentioned that KTableValueGetterSupplier
> is NOT used in IQ, and that a materialized physical store is always used
> today. Yes that is true, and we do have plans to optimize this case soon;
> for example, it is still doable with the proposed KIP-182 that we can
> remove the physical materialized store but use KTableValueGetterSupplier to
> read form a up-stream's physical store and apply the optimizations. Another
> example you mentioned is stream-stream join, where each stream is
> physically materialized into a store, we can definitely optimize this in
> the future to remove the physical materialized store but use something
> else, e.g. a in-memory buffer. Such optimizations are NOT blocked by the
> updated public APIs of KIP-182.
One of the big goals of the refactoring at least was to get rid of the 
overloads
to make implementation of new features easier as one has not to take 
care about
all the overloads. Folding 2 Overloads into 1 with a Builder that has 2 
way of beeing build
wont help much here.

Having the DSL express very closely what happens will only help people 
not getting confused.
Having the store overload on every operation is just plain confusing 
right now.

>
>
> 2. One concern you raise that KIP-182 may actually block such
> optimizations, is that if users do specify a StateStoreSupplier then we
> cannot optimize that away. That is true, but that is by design: if user do
> specify a state store supplier in Materialized API, that is equal to say
> "forget about doing any smart things library, just use what I give to you".
> In other words, the above mentioned optimizations can be applied if users
> do not enforce any specific StateStoreSupplier, for example in
>
> public static <K, V, S extends StateStore> Materialized<K, V, S>
> as(final String
> storeName)
>
> i.e. user only provide a store name, which is similar like handler token
> for IQ; then the library still have the freedom to do smart things
> internally which is totally hidden from the users. It is similar to, like
> in RDBMS or some NoSQL stores like in HIVE / Cassandra: the store engine do
> not have the freedom to do those query plan optimizations if users already
> enforce the specs like join ordering, query plan generation rules, etc.
You call the same method with the builder build differently and its 
going todo
different things. That is my definition of unituitive + The code 
internally has to become
dead ugly as it needs to apply these optimisations basically in the same 
method call or
at the place the Builder is evaluated. This just cries for ugly internal 
code. There is no
way this can become pretty

>
>
> 3. About whether it is worthwhile to "not break the fluent interface", a
> key point that we cannot just try to optimize one or two use cases, but
> consider the whole user space, and ask what are the percentage of users
> that may get affected. Note that in the DSL we have overloaded functions
> where Materialized / Joined / other options are NOT needed so for most
> normal users they do not need to worry about the specs at all.
>
> So suppose there are only X% "advanced" users who would want to enforce
> some state store suppliers, and Y% who like to use IQ, 100-X-Y percent of
> normal users see no difference in terms of programming for either of these
> two approaches: whether to separate the specs into a different set of APIs.
> And for the Y percent of users they are most likely to just use the
> simplest APIs which is `operator(..., Materialized.as(storeName))` which
> does not sound too bad as to `table = operator(...);
> table.materialize(storeName)`. In other words we use the first approach
> then only X percent of users may have an awkward programming with complex
> option specs along with the operator; if we use the second approach the X+Y
> users need to break its programing fluency to call `table.materialize`
> separately. And my personal guess is that
>
> 0 < X << Y < 1, and that X is very minor compared to Y. That is why I feel
> this is a good trade-off.
>
The keypoint here is that It doesn't matter. Any sufficiently usefull 
topology
will get broken up by the user anyways to keep his code sane. And it 
will most
likely be broken up in the interesting parts where IQ would be usefull. 
The math of
how many people are affected by this is therefore not important. 
Additionally
the comparission doesnt make sense as the X-group can still go with a 
fluent
interface only the Y-people need to break their fluent interface


> Guozhang
>
>
> On Fri, Aug 4, 2017 at 6:18 AM, Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Hi Guozhang,
>>
>>   thank you very much for the reply. It explained a lot more of your
>> reasoning to me
>> once again!
>>
>> I have to disagree with you on the first point. As you mentioned the Join
>> Case.
>> A Join is usually a "logically" materialized table and its
>> KTableValueGetterSupplier
>> is to be used when one wants todo a lookup. But this is not at all what is
>> currently
>> Happening. The join merge processor currently maintains its own new
>> statestore
>> when join is invoked with Storename or supplier.
>>
>> This describes the Issue I want to address perfectly. A Joined-Table
>> doesn't become
>> querieable because it is a JOINEDtable but because it is a joinedTABLE.
>> the emphasis here
>> is that we put the store logic with the join and not the table. It is part
>> of the join() method invocation and not the KTable Interface. This
>> abstraction is wrong.
>>
>> This will always show its ugly face. Lets check your example:
>>
>> stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
>> this resulted KTable is materialized in order to complete the aggregation
>> operation
>>    .filter(Materialized.as("store2"))
>>                 // this restuled KTable is not materialized but its
>> GetterSupplier is implemented to get values from "store1"
>>
>> Currently this is only half true. For IQ a store is used that is maintained
>> by the KTableFilterProcessor, for downstream gets like joins the
>> ValueGetterSupplier is used
>> and indeed uses store1.
>>
>> With the String overload (that you picked here on purpose I guess) it
>> works easier
>> as you can logically map those. But with the StateStoreSupplier it
>> wouldn't.
>> you could not optimize this away as the user is expecting puts and gets to
>> be called
>> on what he supplied.
>>
>> table1.filter(() -> true, InMemoryStore).filter(()->true,SQlLiteStore)
>>
>> There is no way to optimize these away.
>> The same argument with the join holds for filter. Its not querrieable
>> because it got filtered
>> it is querrieable because its a KTable. That's where the emphasis needs to
>> go.
>>
>> The second point was new to me. So I had to think about this in more
>> detail.
>> For me the breaking of the flow comes in very natural.
>>
>> One Stream app I put most of my heart in has the these key metrics:
>> It has:
>> 8   input topics.
>> 3   1:n Joins
>> 6   Group bys
>> 2   built in Joins
>> 2   built in left joins
>> some filters and mappers.
>>
>> this is spanning 390 lines, counting java imports and some more stuff.
>>
>> The whole topology forms a tree in wich the input topics usually get
>> joined and then collected to maps
>> and then joined again and collected to maps again. until they get send to
>> 1 final output topic for consumption in our application servers.
>>
>> I would argue it is impossible to express this topology as a chain of
>> calls. What happened is that
>> usually each join + groupBy tuple became its method taking in the builder
>> and return the Table
>> expressing the result of the sub topology. All Ktables that meet each
>> other with the same key in the
>> process get joined (most of that happening on the top level). This leads
>> to breaking in the fluent interface
>> quite naturally. especially if you have 2 KTables expressing
>> sub-topologies joined together. One subtopology had to go into the method
>> call which is unreasonable IMHO.
>>
>> Even inside these methods we broke the chains. The variable names we used
>> give intermediate KTables really helped in making the semantics clear. They
>> are much like CTE's in hive or the required name in Mysql Subquerries. They
>> help to mark milestones inside the topology.
>>
>> I would argue that for big topologies. (I haven't seen others but I think
>> its big) these milestones would
>> be the most important ones for IQ aswell. So i would argue breaking the
>> chains is not really a problem in
>> reality and it can help in many cases. As I laid out, we broke our chained
>> calls intuitively and it helped
>> other developers debugging the logic a lot. Even without detailed streams
>> understanding.
>>
>> If one really do not want to stop the flow. I could argue that one could
>> either do something like this
>>
>> KTable joinresult;
>> KTable<Integer,Integer> t1 = b.table("laa");
>> KTable<Integer,Integer> t2 = b.table("luu");
>> (joinresult = t1.join(t2, (value1, value2) -> value1 + value2))
>> .filter((key, value) -> false);
>>
>> or write a little snitch like that
>>
>> KTable<Integer,Integer> rememberTableandContinue(KTable<Integer,Integer>
>> t){
>>                  joinresult = t;
>>                  return t;
>> }
>>
>> for usuage as such
>>
>> rememberTableandContinue(t1.join(t2, (value1, value2) -> value1 + value2))
>>                          .filter((key, value) -> false);
>>
>> These suggestions might not looks so pretty. But in the context of
>> breaking bigger topology at milestones.
>> I think everything becomes acceptable really. Probably user would store
>> that intermediate  KTable anyways just for clarity.
>>
>> To summarize to give a KTable a name. I would always opt to the host
>> language variable names.
>> Tables used for IQ are probably tables that are of some sort more
>> important to the topology than
>> others and saving them separatly will increase the readability of
>> topologies by a lot IMO.
>>
>> For these quick example Topologies that we have floating around in all
>> places:
>> I am pretty sure one can go unbroken on them and usually the last table
>> will be the one that
>> is needed for IQ then.
>>
>>
>> Thanks again. The second point really got me thinking, as your perspective
>> on the importance
>> of "not break the fluent interface" was not clear to me. I hope I managed
>> to line out why I
>> think it shouldn't have such a big weight in the discussion.
>>
>> PS.: check out Hive CTE, everyone loves them and our Analytic team is
>> crazy for them
>> because you can name them and that brings clarity. and you get rid of the
>> nesting and can
>> split everything into logical chunks of SQL. KTable variables are the CTE
>> of kafka streams.
>> One can probably sell this to people :)
>>
>> Best Jan
>> Enjoyed your feedback! hope mine makes sense
>>
>>
>>
>>
>>
>> On 03.08.2017 00:10, Guozhang Wang wrote:
>>
>>> Hello Jan,
>>>
>>> Thanks for your proposal. As Bill mentioned the main difference is that we
>>> extract the user-customizable materialization logic out of the topology
>>> building DSL workflow. And the main motivations are in two folds:
>>>
>>> 1) efficiency wise, it allows some KTables to not be materialized if
>>> unnecessary, saving one state store instance and changelog topic.
>>>
>>> 2) programming wise, it looks nicer to separate the topology construction
>>> code from the KTable materialization for IQ uses code.
>>>
>>>
>>> Here are my thoughts regarding these two points:
>>>
>>> Regarding 1), I think with whichever the public APIs (either Damian's
>>> proposal or yours), we can always apply the internal optimization to not
>>> physically materialize the KTable. You can take a look at the internal
>>> interface of "KTableValueGetterSupplier", which is used exactly for this
>>> purposes such that a get call on a "logically" materialized KTable can be
>>> traced back to its parent KTables that are physically materialized in a
>>> state store. So following proposed APIs, for example:
>>>
>>>
>>> stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
>>> this resulted KTable is materialized in order to complete the aggregation
>>> operation
>>>                                       .filter(Materialized.as("store2"))
>>>                  // this restuled KTable is not materialized but its
>>> GetterSupplier is implemented to get values from "store1"
>>>
>>>
>>> Or
>>>
>>> table1 = stream.groupByKey(..).aggregate(..);
>>> table2 = table1.filter();
>>>
>>> tabel1.queryHandle("store1");       // this resulted KTable is
>>> materialized
>>> in order to complete the aggregation operation
>>> tabel1.queryHandle("store2")        // this restuled KTable is not
>>> materialized but its GetterSupplier is implemented to get values from
>>> "store1"
>>>
>>>
>>>
>>> When user query a value for "store2" which is not actually materialized
>>> into a state store, the GetterSupplier will be triggered to in turn query
>>> the store for "store1", and then apply the filter operator on-the-fly to
>>> return the value. So the bottom line is, we can achieve the same
>>> efficiency
>>> optimization with either of the public APIs.
>>>
>>>
>>> Regarding 2), I actually have proposed a similar API to yours earlier in
>>> this discussion thread:
>>>
>>> ---------------------------------------
>>>
>>> // specifying the topology, should be concise and conveniently
>>> concatenated, no specs of materialization at all
>>>
>>> KStream stream1 = builder.stream();
>>> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
>>> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
>>> supplier here any more
>>>
>>> // additional code to the topology above, could be more prescriptive
>>> than descriptive
>>> // only advanced users would want to code in both parts above; while other
>>> users would only code the topology as above.
>>>
>>> table1.materialize("queryableStoreName"); // or..
>>> table1.materialize("queryableStoreName").enableCaching().enableLogging();
>>> // or..
>>> table1.materialize(stateStoreSupplier); // we check type (key-value
>>> types,
>>> windowed or not etc) at starting time and add the metrics / logging /
>>> caching / windowing wrapper on top of the store, or..
>>> table1.materialize(stateStoreSupplier).enableCaching().enableLogging();
>>> //
>>> etc..
>>>
>>> ---------------------------------------
>>>
>>> But one caveat of that, as illustrated above, is that you need to have
>>> separate object of the KTable in order to call either "queryHandle" or
>>> "materialize" (whatever the function name is) for the specifications of
>>> materialization options. This can break the concatenation of the topology
>>> construction part of the code, that you cannot simply add one operator
>>> directly after another. So I think this is a trade-off we have to make and
>>> the current approach looks better in this regard.
>>>
>>>
>>>
>>> Guozhang
>>>
>>>
>>>
>>>
>>> On Wed, Aug 2, 2017 at 2:07 PM, Jan Filipiak<Ja...@trivago.com>
>>> wrote:
>>>
>>> Hi Bill,
>>>> totally! So in the original discussion it was mentioned that the
>>>> overloads
>>>> are nasty when implementing new features. So we wanted to get rid of
>>>> them.
>>>> But what I felt was that the
>>>> copy & pasted code in the KTableProcessors for maintaining IQ stores was
>>>> as big as a hurdle as the overloads.
>>>>
>>>> With this proposal I try to shift things into the direction of getting IQ
>>>> for free if
>>>> KTableValueGetterSupplier is properly implemented (like getting join for
>>>> free then). Instead of having the code for maintaining IQ stores all the
>>>> places. I realized I can do that while getting rid of the overloads, that
>>>> makes me feel my proposal is superior.
>>>>
>>>> Further I try to optimize by using as few stores as possible to give the
>>>> user what he needs. That should save all sorts of resources while
>>>> allowing
>>>> faster rebalances.
>>>>
>>>> The target ultimately is to only have KTableSource and the Aggregators
>>>> maintain a Store and provide a ValueGetterSupplier.
>>>>
>>>> Does this makes sense to you?
>>>>
>>>> Best Jan
>>>>
>>>> On 02.08.2017 18:09, Bill Bejeck wrote:
>>>>
>>>> Hi Jan,
>>>>> Thanks for the effort in putting your thoughts down on paper.
>>>>>
>>>>> Comparing what I see from your proposal and what is presented in
>>>>> KIP-182,
>>>>> one of the main differences is the exclusion of an`Materialized`
>>>>> instance
>>>>> in the `KTable` methods.
>>>>>
>>>>> Can you go into more detail why this is so and the specific problems is
>>>>> avoids and or solves with this approach?
>>>>>
>>>>> Thanks!
>>>>> Bill
>>>>>
>>>>> On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <damian.guy@gmail.com
>>>>> <mailto:
>>>>> damian.guy@gmail.com>> wrote:
>>>>>
>>>>>       Hi Jan,
>>>>>
>>>>>       Thanks for taking the time to put this together, appreciated. For
>>>>> the
>>>>>       benefit of others would you mind explaining a bit about your
>>>>>       motivation?
>>>>>
>>>>>       Cheers,
>>>>>       Damian
>>>>>
>>>>>       On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Jan.Filipiak@trivago.com
>>>>>       <ma...@trivago.com>> wrote:
>>>>>
>>>>>       > Hi all,
>>>>>       >
>>>>>       > after some further discussions, the best thing to show my Idea
>>>>>       of how it
>>>>>       > should evolve would be a bigger mock/interface description.
>>>>>       > The goal is to reduce the store maintaining processors to only
>>>>> the
>>>>>       > Aggregators + and KTableSource. While having KTableSource
>>>>> optionally
>>>>>       > materialized.
>>>>>       >
>>>>>       > Introducing KTable:copy() will allow users to maintain state
>>>>>       twice if
>>>>>       > they really want to. KStream::join*() wasn't touched. I never
>>>>>       personally
>>>>>       > used that so I didn't feel
>>>>>       > comfortable enough touching it. Currently still making up my
>>>>>       mind. None
>>>>>       > of the suggestions made it querieable so far. Gouzhangs
>>>>>       'Buffered' idea
>>>>>       > seems ideal here.
>>>>>       >
>>>>>       > please have a look. Looking forward for your opinions.
>>>>>       >
>>>>>       > Best Jan
>>>>>       >
>>>>>       >
>>>>>       >
>>>>>       > On 21.06.2017 17 <tel:21.06.2017%2017>:24, Eno Thereska wrote:
>>>>>       > > (cc’ing user-list too)
>>>>>       > >
>>>>>       > > Given that we already have StateStoreSuppliers that are
>>>>>       configurable
>>>>>       > using the fluent-like API, probably it’s worth discussing the
>>>>> other
>>>>>       > examples with joins and serdes first since those have many
>>>>>       overloads and
>>>>>       > are in need of some TLC.
>>>>>       > >
>>>>>       > > So following your example, I guess you’d have something like:
>>>>>       > > .join()
>>>>>       > >     .withKeySerdes(…)
>>>>>       > >     .withValueSerdes(…)
>>>>>       > >     .withJoinType(“outer”)
>>>>>       > >
>>>>>       > > etc?
>>>>>       > >
>>>>>       > > I like the approach since it still remains declarative and
>>>>>       it’d reduce
>>>>>       > the number of overloads by quite a bit.
>>>>>       > >
>>>>>       > > Eno
>>>>>       > >
>>>>>       > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>>>       <ma...@gmail.com>> wrote:
>>>>>       > >>
>>>>>       > >> Hi,
>>>>>       > >>
>>>>>       > >> I'd like to get a discussion going around some of the API
>>>>>       choices we've
>>>>>       > >> made in the DLS. In particular those that relate to stateful
>>>>>       operations
>>>>>       > >> (though this could expand).
>>>>>       > >> As it stands we lean heavily on overloaded methods in the
>>>>>       API, i.e,
>>>>>       > there
>>>>>       > >> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>>>       noisy and i
>>>>>       > >> feel it is only going to get worse as we add more optional
>>>>>       params. In
>>>>>       > >> particular we've had some requests to be able to turn caching
>>>>>       off, or
>>>>>       > >> change log configs,  on a per operator basis (note this can
>>>>>       be done now
>>>>>       > if
>>>>>       > >> you pass in a StateStoreSupplier, but this can be a bit
>>>>>       cumbersome).
>>>>>       > >>
>>>>>       > >> So this is a bit of an open question. How can we change the
>>>>> DSL
>>>>>       > overloads
>>>>>       > >> so that it flows, is simple to use and understand, and is
>>>>> easily
>>>>>       > extended
>>>>>       > >> in the future?
>>>>>       > >>
>>>>>       > >> One option would be to use a fluent API approach for
>>>>>       providing the
>>>>>       > optional
>>>>>       > >> params, so something like this:
>>>>>       > >>
>>>>>       > >> groupedStream.count()
>>>>>       > >>    .withStoreName("name")
>>>>>       > >>    .withCachingEnabled(false)
>>>>>       > >>    .withLoggingEnabled(config)
>>>>>       > >>    .table()
>>>>>       > >>
>>>>>       > >>
>>>>>       > >>
>>>>>       > >> Another option would be to provide a Builder to the count
>>>>>       method, so it
>>>>>       > >> would look something like this:
>>>>>       > >> groupedStream.count(new
>>>>>       > >> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>       > >>
>>>>>       > >> Another option is to say: Hey we don't need this, what are
>>>>>       you on about!
>>>>>       > >>
>>>>>       > >> The above has focussed on state store related overloads, but
>>>>>       the same
>>>>>       > ideas
>>>>>       > >> could  be applied to joins etc, where we presently have many
>>>>> join
>>>>>       > methods
>>>>>       > >> and many overloads.
>>>>>       > >>
>>>>>       > >> Anyway, i look forward to hearing your opinions.
>>>>>       > >>
>>>>>       > >> Thanks,
>>>>>       > >> Damian
>>>>>       >
>>>>>       >
>>>>>
>>>>>
>>>>>
>>>>>
>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jan,

Thanks for your feedback. Trying to explain them a bit more here since I
think there are still a bit mis-communication here:

Here are a few things I need to clarify for KIP-182 first:

1. KIP-182 is mainly about refactoring the public APIs, NOT for making any
optimizations on the internal implementations. So we only care that these
public APIs changes do not forbid us to make the internal implementations
in the near future.

To give you a concrete example, as you mentioned that KTableValueGetterSupplier
is NOT used in IQ, and that a materialized physical store is always used
today. Yes that is true, and we do have plans to optimize this case soon;
for example, it is still doable with the proposed KIP-182 that we can
remove the physical materialized store but use KTableValueGetterSupplier to
read form a up-stream's physical store and apply the optimizations. Another
example you mentioned is stream-stream join, where each stream is
physically materialized into a store, we can definitely optimize this in
the future to remove the physical materialized store but use something
else, e.g. a in-memory buffer. Such optimizations are NOT blocked by the
updated public APIs of KIP-182.


2. One concern you raise that KIP-182 may actually block such
optimizations, is that if users do specify a StateStoreSupplier then we
cannot optimize that away. That is true, but that is by design: if user do
specify a state store supplier in Materialized API, that is equal to say
"forget about doing any smart things library, just use what I give to you".
In other words, the above mentioned optimizations can be applied if users
do not enforce any specific StateStoreSupplier, for example in

public static <K, V, S extends StateStore> Materialized<K, V, S>
as(final String
storeName)

i.e. user only provide a store name, which is similar like handler token
for IQ; then the library still have the freedom to do smart things
internally which is totally hidden from the users. It is similar to, like
in RDBMS or some NoSQL stores like in HIVE / Cassandra: the store engine do
not have the freedom to do those query plan optimizations if users already
enforce the specs like join ordering, query plan generation rules, etc.


3. About whether it is worthwhile to "not break the fluent interface", a
key point that we cannot just try to optimize one or two use cases, but
consider the whole user space, and ask what are the percentage of users
that may get affected. Note that in the DSL we have overloaded functions
where Materialized / Joined / other options are NOT needed so for most
normal users they do not need to worry about the specs at all.

So suppose there are only X% "advanced" users who would want to enforce
some state store suppliers, and Y% who like to use IQ, 100-X-Y percent of
normal users see no difference in terms of programming for either of these
two approaches: whether to separate the specs into a different set of APIs.
And for the Y percent of users they are most likely to just use the
simplest APIs which is `operator(..., Materialized.as(storeName))` which
does not sound too bad as to `table = operator(...);
table.materialize(storeName)`. In other words we use the first approach
then only X percent of users may have an awkward programming with complex
option specs along with the operator; if we use the second approach the X+Y
users need to break its programing fluency to call `table.materialize`
separately. And my personal guess is that

0 < X << Y < 1, and that X is very minor compared to Y. That is why I feel
this is a good trade-off.


Guozhang


On Fri, Aug 4, 2017 at 6:18 AM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi Guozhang,
>
>  thank you very much for the reply. It explained a lot more of your
> reasoning to me
> once again!
>
> I have to disagree with you on the first point. As you mentioned the Join
> Case.
> A Join is usually a "logically" materialized table and its
> KTableValueGetterSupplier
> is to be used when one wants todo a lookup. But this is not at all what is
> currently
> Happening. The join merge processor currently maintains its own new
> statestore
> when join is invoked with Storename or supplier.
>
> This describes the Issue I want to address perfectly. A Joined-Table
> doesn't become
> querieable because it is a JOINEDtable but because it is a joinedTABLE.
> the emphasis here
> is that we put the store logic with the join and not the table. It is part
> of the join() method invocation and not the KTable Interface. This
> abstraction is wrong.
>
> This will always show its ugly face. Lets check your example:
>
> stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
> this resulted KTable is materialized in order to complete the aggregation
> operation
>   .filter(Materialized.as("store2"))
>                // this restuled KTable is not materialized but its
> GetterSupplier is implemented to get values from "store1"
>
> Currently this is only half true. For IQ a store is used that is maintained
> by the KTableFilterProcessor, for downstream gets like joins the
> ValueGetterSupplier is used
> and indeed uses store1.
>
> With the String overload (that you picked here on purpose I guess) it
> works easier
> as you can logically map those. But with the StateStoreSupplier it
> wouldn't.
> you could not optimize this away as the user is expecting puts and gets to
> be called
> on what he supplied.
>
> table1.filter(() -> true, InMemoryStore).filter(()->true,SQlLiteStore)
>
> There is no way to optimize these away.
> The same argument with the join holds for filter. Its not querrieable
> because it got filtered
> it is querrieable because its a KTable. That's where the emphasis needs to
> go.
>
> The second point was new to me. So I had to think about this in more
> detail.
> For me the breaking of the flow comes in very natural.
>
> One Stream app I put most of my heart in has the these key metrics:
> It has:
> 8   input topics.
> 3   1:n Joins
> 6   Group bys
> 2   built in Joins
> 2   built in left joins
> some filters and mappers.
>
> this is spanning 390 lines, counting java imports and some more stuff.
>
> The whole topology forms a tree in wich the input topics usually get
> joined and then collected to maps
> and then joined again and collected to maps again. until they get send to
> 1 final output topic for consumption in our application servers.
>
> I would argue it is impossible to express this topology as a chain of
> calls. What happened is that
> usually each join + groupBy tuple became its method taking in the builder
> and return the Table
> expressing the result of the sub topology. All Ktables that meet each
> other with the same key in the
> process get joined (most of that happening on the top level). This leads
> to breaking in the fluent interface
> quite naturally. especially if you have 2 KTables expressing
> sub-topologies joined together. One subtopology had to go into the method
> call which is unreasonable IMHO.
>
> Even inside these methods we broke the chains. The variable names we used
> give intermediate KTables really helped in making the semantics clear. They
> are much like CTE's in hive or the required name in Mysql Subquerries. They
> help to mark milestones inside the topology.
>
> I would argue that for big topologies. (I haven't seen others but I think
> its big) these milestones would
> be the most important ones for IQ aswell. So i would argue breaking the
> chains is not really a problem in
> reality and it can help in many cases. As I laid out, we broke our chained
> calls intuitively and it helped
> other developers debugging the logic a lot. Even without detailed streams
> understanding.
>
> If one really do not want to stop the flow. I could argue that one could
> either do something like this
>
> KTable joinresult;
> KTable<Integer,Integer> t1 = b.table("laa");
> KTable<Integer,Integer> t2 = b.table("luu");
> (joinresult = t1.join(t2, (value1, value2) -> value1 + value2))
> .filter((key, value) -> false);
>
> or write a little snitch like that
>
> KTable<Integer,Integer> rememberTableandContinue(KTable<Integer,Integer>
> t){
>                 joinresult = t;
>                 return t;
> }
>
> for usuage as such
>
> rememberTableandContinue(t1.join(t2, (value1, value2) -> value1 + value2))
>                         .filter((key, value) -> false);
>
> These suggestions might not looks so pretty. But in the context of
> breaking bigger topology at milestones.
> I think everything becomes acceptable really. Probably user would store
> that intermediate  KTable anyways just for clarity.
>
> To summarize to give a KTable a name. I would always opt to the host
> language variable names.
> Tables used for IQ are probably tables that are of some sort more
> important to the topology than
> others and saving them separatly will increase the readability of
> topologies by a lot IMO.
>
> For these quick example Topologies that we have floating around in all
> places:
> I am pretty sure one can go unbroken on them and usually the last table
> will be the one that
> is needed for IQ then.
>
>
> Thanks again. The second point really got me thinking, as your perspective
> on the importance
> of "not break the fluent interface" was not clear to me. I hope I managed
> to line out why I
> think it shouldn't have such a big weight in the discussion.
>
> PS.: check out Hive CTE, everyone loves them and our Analytic team is
> crazy for them
> because you can name them and that brings clarity. and you get rid of the
> nesting and can
> split everything into logical chunks of SQL. KTable variables are the CTE
> of kafka streams.
> One can probably sell this to people :)
>
> Best Jan
> Enjoyed your feedback! hope mine makes sense
>
>
>
>
>
> On 03.08.2017 00:10, Guozhang Wang wrote:
>
>> Hello Jan,
>>
>> Thanks for your proposal. As Bill mentioned the main difference is that we
>> extract the user-customizable materialization logic out of the topology
>> building DSL workflow. And the main motivations are in two folds:
>>
>> 1) efficiency wise, it allows some KTables to not be materialized if
>> unnecessary, saving one state store instance and changelog topic.
>>
>> 2) programming wise, it looks nicer to separate the topology construction
>> code from the KTable materialization for IQ uses code.
>>
>>
>> Here are my thoughts regarding these two points:
>>
>> Regarding 1), I think with whichever the public APIs (either Damian's
>> proposal or yours), we can always apply the internal optimization to not
>> physically materialize the KTable. You can take a look at the internal
>> interface of "KTableValueGetterSupplier", which is used exactly for this
>> purposes such that a get call on a "logically" materialized KTable can be
>> traced back to its parent KTables that are physically materialized in a
>> state store. So following proposed APIs, for example:
>>
>>
>> stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
>> this resulted KTable is materialized in order to complete the aggregation
>> operation
>>                                      .filter(Materialized.as("store2"))
>>                 // this restuled KTable is not materialized but its
>> GetterSupplier is implemented to get values from "store1"
>>
>>
>> Or
>>
>> table1 = stream.groupByKey(..).aggregate(..);
>> table2 = table1.filter();
>>
>> tabel1.queryHandle("store1");       // this resulted KTable is
>> materialized
>> in order to complete the aggregation operation
>> tabel1.queryHandle("store2")        // this restuled KTable is not
>> materialized but its GetterSupplier is implemented to get values from
>> "store1"
>>
>>
>>
>> When user query a value for "store2" which is not actually materialized
>> into a state store, the GetterSupplier will be triggered to in turn query
>> the store for "store1", and then apply the filter operator on-the-fly to
>> return the value. So the bottom line is, we can achieve the same
>> efficiency
>> optimization with either of the public APIs.
>>
>>
>> Regarding 2), I actually have proposed a similar API to yours earlier in
>> this discussion thread:
>>
>> ---------------------------------------
>>
>> // specifying the topology, should be concise and conveniently
>> concatenated, no specs of materialization at all
>>
>> KStream stream1 = builder.stream();
>> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
>> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
>> supplier here any more
>>
>> // additional code to the topology above, could be more prescriptive
>> than descriptive
>> // only advanced users would want to code in both parts above; while other
>> users would only code the topology as above.
>>
>> table1.materialize("queryableStoreName"); // or..
>> table1.materialize("queryableStoreName").enableCaching().enableLogging();
>> // or..
>> table1.materialize(stateStoreSupplier); // we check type (key-value
>> types,
>> windowed or not etc) at starting time and add the metrics / logging /
>> caching / windowing wrapper on top of the store, or..
>> table1.materialize(stateStoreSupplier).enableCaching().enableLogging();
>> //
>> etc..
>>
>> ---------------------------------------
>>
>> But one caveat of that, as illustrated above, is that you need to have
>> separate object of the KTable in order to call either "queryHandle" or
>> "materialize" (whatever the function name is) for the specifications of
>> materialization options. This can break the concatenation of the topology
>> construction part of the code, that you cannot simply add one operator
>> directly after another. So I think this is a trade-off we have to make and
>> the current approach looks better in this regard.
>>
>>
>>
>> Guozhang
>>
>>
>>
>>
>> On Wed, Aug 2, 2017 at 2:07 PM, Jan Filipiak<Ja...@trivago.com>
>> wrote:
>>
>> Hi Bill,
>>>
>>> totally! So in the original discussion it was mentioned that the
>>> overloads
>>> are nasty when implementing new features. So we wanted to get rid of
>>> them.
>>> But what I felt was that the
>>> copy & pasted code in the KTableProcessors for maintaining IQ stores was
>>> as big as a hurdle as the overloads.
>>>
>>> With this proposal I try to shift things into the direction of getting IQ
>>> for free if
>>> KTableValueGetterSupplier is properly implemented (like getting join for
>>> free then). Instead of having the code for maintaining IQ stores all the
>>> places. I realized I can do that while getting rid of the overloads, that
>>> makes me feel my proposal is superior.
>>>
>>> Further I try to optimize by using as few stores as possible to give the
>>> user what he needs. That should save all sorts of resources while
>>> allowing
>>> faster rebalances.
>>>
>>> The target ultimately is to only have KTableSource and the Aggregators
>>> maintain a Store and provide a ValueGetterSupplier.
>>>
>>> Does this makes sense to you?
>>>
>>> Best Jan
>>>
>>> On 02.08.2017 18:09, Bill Bejeck wrote:
>>>
>>> Hi Jan,
>>>>
>>>> Thanks for the effort in putting your thoughts down on paper.
>>>>
>>>> Comparing what I see from your proposal and what is presented in
>>>> KIP-182,
>>>> one of the main differences is the exclusion of an`Materialized`
>>>> instance
>>>> in the `KTable` methods.
>>>>
>>>> Can you go into more detail why this is so and the specific problems is
>>>> avoids and or solves with this approach?
>>>>
>>>> Thanks!
>>>> Bill
>>>>
>>>> On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <damian.guy@gmail.com
>>>> <mailto:
>>>> damian.guy@gmail.com>> wrote:
>>>>
>>>>      Hi Jan,
>>>>
>>>>      Thanks for taking the time to put this together, appreciated. For
>>>> the
>>>>      benefit of others would you mind explaining a bit about your
>>>>      motivation?
>>>>
>>>>      Cheers,
>>>>      Damian
>>>>
>>>>      On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Jan.Filipiak@trivago.com
>>>>      <ma...@trivago.com>> wrote:
>>>>
>>>>      > Hi all,
>>>>      >
>>>>      > after some further discussions, the best thing to show my Idea
>>>>      of how it
>>>>      > should evolve would be a bigger mock/interface description.
>>>>      > The goal is to reduce the store maintaining processors to only
>>>> the
>>>>      > Aggregators + and KTableSource. While having KTableSource
>>>> optionally
>>>>      > materialized.
>>>>      >
>>>>      > Introducing KTable:copy() will allow users to maintain state
>>>>      twice if
>>>>      > they really want to. KStream::join*() wasn't touched. I never
>>>>      personally
>>>>      > used that so I didn't feel
>>>>      > comfortable enough touching it. Currently still making up my
>>>>      mind. None
>>>>      > of the suggestions made it querieable so far. Gouzhangs
>>>>      'Buffered' idea
>>>>      > seems ideal here.
>>>>      >
>>>>      > please have a look. Looking forward for your opinions.
>>>>      >
>>>>      > Best Jan
>>>>      >
>>>>      >
>>>>      >
>>>>      > On 21.06.2017 17 <tel:21.06.2017%2017>:24, Eno Thereska wrote:
>>>>      > > (cc’ing user-list too)
>>>>      > >
>>>>      > > Given that we already have StateStoreSuppliers that are
>>>>      configurable
>>>>      > using the fluent-like API, probably it’s worth discussing the
>>>> other
>>>>      > examples with joins and serdes first since those have many
>>>>      overloads and
>>>>      > are in need of some TLC.
>>>>      > >
>>>>      > > So following your example, I guess you’d have something like:
>>>>      > > .join()
>>>>      > >     .withKeySerdes(…)
>>>>      > >     .withValueSerdes(…)
>>>>      > >     .withJoinType(“outer”)
>>>>      > >
>>>>      > > etc?
>>>>      > >
>>>>      > > I like the approach since it still remains declarative and
>>>>      it’d reduce
>>>>      > the number of overloads by quite a bit.
>>>>      > >
>>>>      > > Eno
>>>>      > >
>>>>      > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>>      <ma...@gmail.com>> wrote:
>>>>      > >>
>>>>      > >> Hi,
>>>>      > >>
>>>>      > >> I'd like to get a discussion going around some of the API
>>>>      choices we've
>>>>      > >> made in the DLS. In particular those that relate to stateful
>>>>      operations
>>>>      > >> (though this could expand).
>>>>      > >> As it stands we lean heavily on overloaded methods in the
>>>>      API, i.e,
>>>>      > there
>>>>      > >> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>>      noisy and i
>>>>      > >> feel it is only going to get worse as we add more optional
>>>>      params. In
>>>>      > >> particular we've had some requests to be able to turn caching
>>>>      off, or
>>>>      > >> change log configs,  on a per operator basis (note this can
>>>>      be done now
>>>>      > if
>>>>      > >> you pass in a StateStoreSupplier, but this can be a bit
>>>>      cumbersome).
>>>>      > >>
>>>>      > >> So this is a bit of an open question. How can we change the
>>>> DSL
>>>>      > overloads
>>>>      > >> so that it flows, is simple to use and understand, and is
>>>> easily
>>>>      > extended
>>>>      > >> in the future?
>>>>      > >>
>>>>      > >> One option would be to use a fluent API approach for
>>>>      providing the
>>>>      > optional
>>>>      > >> params, so something like this:
>>>>      > >>
>>>>      > >> groupedStream.count()
>>>>      > >>    .withStoreName("name")
>>>>      > >>    .withCachingEnabled(false)
>>>>      > >>    .withLoggingEnabled(config)
>>>>      > >>    .table()
>>>>      > >>
>>>>      > >>
>>>>      > >>
>>>>      > >> Another option would be to provide a Builder to the count
>>>>      method, so it
>>>>      > >> would look something like this:
>>>>      > >> groupedStream.count(new
>>>>      > >> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>      > >>
>>>>      > >> Another option is to say: Hey we don't need this, what are
>>>>      you on about!
>>>>      > >>
>>>>      > >> The above has focussed on state store related overloads, but
>>>>      the same
>>>>      > ideas
>>>>      > >> could  be applied to joins etc, where we presently have many
>>>> join
>>>>      > methods
>>>>      > >> and many overloads.
>>>>      > >>
>>>>      > >> Anyway, i look forward to hearing your opinions.
>>>>      > >>
>>>>      > >> Thanks,
>>>>      > >> Damian
>>>>      >
>>>>      >
>>>>
>>>>
>>>>
>>>>
>


-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Guozhang,

  thank you very much for the reply. It explained a lot more of your 
reasoning to me
once again!

I have to disagree with you on the first point. As you mentioned the 
Join Case.
A Join is usually a "logically" materialized table and its 
KTableValueGetterSupplier
is to be used when one wants todo a lookup. But this is not at all what 
is currently
Happening. The join merge processor currently maintains its own new 
statestore
when join is invoked with Storename or supplier.

This describes the Issue I want to address perfectly. A Joined-Table 
doesn't become
querieable because it is a JOINEDtable but because it is a joinedTABLE.  
the emphasis here
is that we put the store logic with the join and not the table. It is 
part of the join() method invocation and not the KTable Interface. This 
abstraction is wrong.

This will always show its ugly face. Lets check your example:

stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
this resulted KTable is materialized in order to complete the aggregation
operation
   .filter(Materialized.as("store2"))
                // this restuled KTable is not materialized but its
GetterSupplier is implemented to get values from "store1"

Currently this is only half true. For IQ a store is used that is maintained
by the KTableFilterProcessor, for downstream gets like joins the ValueGetterSupplier is used
and indeed uses store1.

With the String overload (that you picked here on purpose I guess) it works easier
as you can logically map those. But with the StateStoreSupplier it wouldn't.
you could not optimize this away as the user is expecting puts and gets to be called
on what he supplied.

table1.filter(() -> true, InMemoryStore).filter(()->true,SQlLiteStore)

There is no way to optimize these away.
The same argument with the join holds for filter. Its not querrieable because it got filtered
it is querrieable because its a KTable. That's where the emphasis needs to go.

The second point was new to me. So I had to think about this in more detail.
For me the breaking of the flow comes in very natural.

One Stream app I put most of my heart in has the these key metrics:
It has:
8   input topics.
3   1:n Joins
6   Group bys
2   built in Joins
2   built in left joins
some filters and mappers.

this is spanning 390 lines, counting java imports and some more stuff.

The whole topology forms a tree in wich the input topics usually get joined and then collected to maps
and then joined again and collected to maps again. until they get send to 1 final output topic for consumption in our application servers.

I would argue it is impossible to express this topology as a chain of calls. What happened is that
usually each join + groupBy tuple became its method taking in the builder and return the Table
expressing the result of the sub topology. All Ktables that meet each other with the same key in the
process get joined (most of that happening on the top level). This leads to breaking in the fluent interface
quite naturally. especially if you have 2 KTables expressing sub-topologies joined together. One subtopology had to go into the method call which is unreasonable IMHO.

Even inside these methods we broke the chains. The variable names we used give intermediate KTables really helped in making the semantics clear. They are much like CTE's in hive or the required name in Mysql Subquerries. They help to mark milestones inside the topology.

I would argue that for big topologies. (I haven't seen others but I think its big) these milestones would
be the most important ones for IQ aswell. So i would argue breaking the chains is not really a problem in
reality and it can help in many cases. As I laid out, we broke our chained calls intuitively and it helped
other developers debugging the logic a lot. Even without detailed streams understanding.

If one really do not want to stop the flow. I could argue that one could either do something like this

KTable joinresult;
KTable<Integer,Integer> t1 = b.table("laa");
KTable<Integer,Integer> t2 = b.table("luu");
(joinresult = t1.join(t2, (value1, value2) -> value1 + value2))
.filter((key, value) -> false);

or write a little snitch like that

KTable<Integer,Integer> rememberTableandContinue(KTable<Integer,Integer> t){
		joinresult = t;
		return t;
}

for usuage as such

rememberTableandContinue(t1.join(t2, (value1, value2) -> value1 + value2))
			.filter((key, value) -> false);

These suggestions might not looks so pretty. But in the context of breaking bigger topology at milestones.
I think everything becomes acceptable really. Probably user would store that intermediate  KTable anyways just for clarity.

To summarize to give a KTable a name. I would always opt to the host language variable names.
Tables used for IQ are probably tables that are of some sort more important to the topology than
others and saving them separatly will increase the readability of topologies by a lot IMO.

For these quick example Topologies that we have floating around in all places:
I am pretty sure one can go unbroken on them and usually the last table will be the one that
is needed for IQ then.


Thanks again. The second point really got me thinking, as your perspective on the importance
of "not break the fluent interface" was not clear to me. I hope I managed to line out why I
think it shouldn't have such a big weight in the discussion.

PS.: check out Hive CTE, everyone loves them and our Analytic team is crazy for them
because you can name them and that brings clarity. and you get rid of the nesting and can
split everything into logical chunks of SQL. KTable variables are the CTE of kafka streams.
One can probably sell this to people :)

Best Jan
Enjoyed your feedback! hope mine makes sense




On 03.08.2017 00:10, Guozhang Wang wrote:
> Hello Jan,
>
> Thanks for your proposal. As Bill mentioned the main difference is that we
> extract the user-customizable materialization logic out of the topology
> building DSL workflow. And the main motivations are in two folds:
>
> 1) efficiency wise, it allows some KTables to not be materialized if
> unnecessary, saving one state store instance and changelog topic.
>
> 2) programming wise, it looks nicer to separate the topology construction
> code from the KTable materialization for IQ uses code.
>
>
> Here are my thoughts regarding these two points:
>
> Regarding 1), I think with whichever the public APIs (either Damian's
> proposal or yours), we can always apply the internal optimization to not
> physically materialize the KTable. You can take a look at the internal
> interface of "KTableValueGetterSupplier", which is used exactly for this
> purposes such that a get call on a "logically" materialized KTable can be
> traced back to its parent KTables that are physically materialized in a
> state store. So following proposed APIs, for example:
>
>
> stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
> this resulted KTable is materialized in order to complete the aggregation
> operation
>                                      .filter(Materialized.as("store2"))
>                 // this restuled KTable is not materialized but its
> GetterSupplier is implemented to get values from "store1"
>
>
> Or
>
> table1 = stream.groupByKey(..).aggregate(..);
> table2 = table1.filter();
>
> tabel1.queryHandle("store1");       // this resulted KTable is materialized
> in order to complete the aggregation operation
> tabel1.queryHandle("store2")        // this restuled KTable is not
> materialized but its GetterSupplier is implemented to get values from
> "store1"
>
>
>
> When user query a value for "store2" which is not actually materialized
> into a state store, the GetterSupplier will be triggered to in turn query
> the store for "store1", and then apply the filter operator on-the-fly to
> return the value. So the bottom line is, we can achieve the same efficiency
> optimization with either of the public APIs.
>
>
> Regarding 2), I actually have proposed a similar API to yours earlier in
> this discussion thread:
>
> ---------------------------------------
>
> // specifying the topology, should be concise and conveniently
> concatenated, no specs of materialization at all
>
> KStream stream1 = builder.stream();
> KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
> sessionMerger, sessionWindows);  // do not allow to pass-in a state store
> supplier here any more
>
> // additional code to the topology above, could be more prescriptive
> than descriptive
> // only advanced users would want to code in both parts above; while other
> users would only code the topology as above.
>
> table1.materialize("queryableStoreName"); // or..
> table1.materialize("queryableStoreName").enableCaching().enableLogging();
> // or..
> table1.materialize(stateStoreSupplier); // we check type (key-value types,
> windowed or not etc) at starting time and add the metrics / logging /
> caching / windowing wrapper on top of the store, or..
> table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
> etc..
>
> ---------------------------------------
>
> But one caveat of that, as illustrated above, is that you need to have
> separate object of the KTable in order to call either "queryHandle" or
> "materialize" (whatever the function name is) for the specifications of
> materialization options. This can break the concatenation of the topology
> construction part of the code, that you cannot simply add one operator
> directly after another. So I think this is a trade-off we have to make and
> the current approach looks better in this regard.
>
>
>
> Guozhang
>
>
>
>
> On Wed, Aug 2, 2017 at 2:07 PM, Jan Filipiak<Ja...@trivago.com>
> wrote:
>
>> Hi Bill,
>>
>> totally! So in the original discussion it was mentioned that the overloads
>> are nasty when implementing new features. So we wanted to get rid of them.
>> But what I felt was that the
>> copy & pasted code in the KTableProcessors for maintaining IQ stores was
>> as big as a hurdle as the overloads.
>>
>> With this proposal I try to shift things into the direction of getting IQ
>> for free if
>> KTableValueGetterSupplier is properly implemented (like getting join for
>> free then). Instead of having the code for maintaining IQ stores all the
>> places. I realized I can do that while getting rid of the overloads, that
>> makes me feel my proposal is superior.
>>
>> Further I try to optimize by using as few stores as possible to give the
>> user what he needs. That should save all sorts of resources while allowing
>> faster rebalances.
>>
>> The target ultimately is to only have KTableSource and the Aggregators
>> maintain a Store and provide a ValueGetterSupplier.
>>
>> Does this makes sense to you?
>>
>> Best Jan
>>
>> On 02.08.2017 18:09, Bill Bejeck wrote:
>>
>>> Hi Jan,
>>>
>>> Thanks for the effort in putting your thoughts down on paper.
>>>
>>> Comparing what I see from your proposal and what is presented in KIP-182,
>>> one of the main differences is the exclusion of an`Materialized`  instance
>>> in the `KTable` methods.
>>>
>>> Can you go into more detail why this is so and the specific problems is
>>> avoids and or solves with this approach?
>>>
>>> Thanks!
>>> Bill
>>>
>>> On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <damian.guy@gmail.com  <mailto:
>>> damian.guy@gmail.com>> wrote:
>>>
>>>      Hi Jan,
>>>
>>>      Thanks for taking the time to put this together, appreciated. For the
>>>      benefit of others would you mind explaining a bit about your
>>>      motivation?
>>>
>>>      Cheers,
>>>      Damian
>>>
>>>      On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Jan.Filipiak@trivago.com
>>>      <ma...@trivago.com>> wrote:
>>>
>>>      > Hi all,
>>>      >
>>>      > after some further discussions, the best thing to show my Idea
>>>      of how it
>>>      > should evolve would be a bigger mock/interface description.
>>>      > The goal is to reduce the store maintaining processors to only the
>>>      > Aggregators + and KTableSource. While having KTableSource optionally
>>>      > materialized.
>>>      >
>>>      > Introducing KTable:copy() will allow users to maintain state
>>>      twice if
>>>      > they really want to. KStream::join*() wasn't touched. I never
>>>      personally
>>>      > used that so I didn't feel
>>>      > comfortable enough touching it. Currently still making up my
>>>      mind. None
>>>      > of the suggestions made it querieable so far. Gouzhangs
>>>      'Buffered' idea
>>>      > seems ideal here.
>>>      >
>>>      > please have a look. Looking forward for your opinions.
>>>      >
>>>      > Best Jan
>>>      >
>>>      >
>>>      >
>>>      > On 21.06.2017 17 <tel:21.06.2017%2017>:24, Eno Thereska wrote:
>>>      > > (cc’ing user-list too)
>>>      > >
>>>      > > Given that we already have StateStoreSuppliers that are
>>>      configurable
>>>      > using the fluent-like API, probably it’s worth discussing the other
>>>      > examples with joins and serdes first since those have many
>>>      overloads and
>>>      > are in need of some TLC.
>>>      > >
>>>      > > So following your example, I guess you’d have something like:
>>>      > > .join()
>>>      > >     .withKeySerdes(…)
>>>      > >     .withValueSerdes(…)
>>>      > >     .withJoinType(“outer”)
>>>      > >
>>>      > > etc?
>>>      > >
>>>      > > I like the approach since it still remains declarative and
>>>      it’d reduce
>>>      > the number of overloads by quite a bit.
>>>      > >
>>>      > > Eno
>>>      > >
>>>      > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>      <ma...@gmail.com>> wrote:
>>>      > >>
>>>      > >> Hi,
>>>      > >>
>>>      > >> I'd like to get a discussion going around some of the API
>>>      choices we've
>>>      > >> made in the DLS. In particular those that relate to stateful
>>>      operations
>>>      > >> (though this could expand).
>>>      > >> As it stands we lean heavily on overloaded methods in the
>>>      API, i.e,
>>>      > there
>>>      > >> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>      noisy and i
>>>      > >> feel it is only going to get worse as we add more optional
>>>      params. In
>>>      > >> particular we've had some requests to be able to turn caching
>>>      off, or
>>>      > >> change log configs,  on a per operator basis (note this can
>>>      be done now
>>>      > if
>>>      > >> you pass in a StateStoreSupplier, but this can be a bit
>>>      cumbersome).
>>>      > >>
>>>      > >> So this is a bit of an open question. How can we change the DSL
>>>      > overloads
>>>      > >> so that it flows, is simple to use and understand, and is easily
>>>      > extended
>>>      > >> in the future?
>>>      > >>
>>>      > >> One option would be to use a fluent API approach for
>>>      providing the
>>>      > optional
>>>      > >> params, so something like this:
>>>      > >>
>>>      > >> groupedStream.count()
>>>      > >>    .withStoreName("name")
>>>      > >>    .withCachingEnabled(false)
>>>      > >>    .withLoggingEnabled(config)
>>>      > >>    .table()
>>>      > >>
>>>      > >>
>>>      > >>
>>>      > >> Another option would be to provide a Builder to the count
>>>      method, so it
>>>      > >> would look something like this:
>>>      > >> groupedStream.count(new
>>>      > >> CountBuilder("storeName").withCachingEnabled(false).build())
>>>      > >>
>>>      > >> Another option is to say: Hey we don't need this, what are
>>>      you on about!
>>>      > >>
>>>      > >> The above has focussed on state store related overloads, but
>>>      the same
>>>      > ideas
>>>      > >> could  be applied to joins etc, where we presently have many join
>>>      > methods
>>>      > >> and many overloads.
>>>      > >>
>>>      > >> Anyway, i look forward to hearing your opinions.
>>>      > >>
>>>      > >> Thanks,
>>>      > >> Damian
>>>      >
>>>      >
>>>
>>>
>>>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jan,

Thanks for your proposal. As Bill mentioned the main difference is that we
extract the user-customizable materialization logic out of the topology
building DSL workflow. And the main motivations are in two folds:

1) efficiency wise, it allows some KTables to not be materialized if
unnecessary, saving one state store instance and changelog topic.

2) programming wise, it looks nicer to separate the topology construction
code from the KTable materialization for IQ uses code.


Here are my thoughts regarding these two points:

Regarding 1), I think with whichever the public APIs (either Damian's
proposal or yours), we can always apply the internal optimization to not
physically materialize the KTable. You can take a look at the internal
interface of "KTableValueGetterSupplier", which is used exactly for this
purposes such that a get call on a "logically" materialized KTable can be
traced back to its parent KTables that are physically materialized in a
state store. So following proposed APIs, for example:


stream.groupByKey(..).aggregate(.., Materializedas("store1"))        //
this resulted KTable is materialized in order to complete the aggregation
operation
                                    .filter(Materialized.as("store2"))
               // this restuled KTable is not materialized but its
GetterSupplier is implemented to get values from "store1"


Or

table1 = stream.groupByKey(..).aggregate(..);
table2 = table1.filter();

tabel1.queryHandle("store1");       // this resulted KTable is materialized
in order to complete the aggregation operation
tabel1.queryHandle("store2")        // this restuled KTable is not
materialized but its GetterSupplier is implemented to get values from
"store1"



When user query a value for "store2" which is not actually materialized
into a state store, the GetterSupplier will be triggered to in turn query
the store for "store1", and then apply the filter operator on-the-fly to
return the value. So the bottom line is, we can achieve the same efficiency
optimization with either of the public APIs.


Regarding 2), I actually have proposed a similar API to yours earlier in
this discussion thread:

---------------------------------------

// specifying the topology, should be concise and conveniently
concatenated, no specs of materialization at all

KStream stream1 = builder.stream();
KTable table1 = stream1.groupby(...).aggregate(initializer, aggregator,
sessionMerger, sessionWindows);  // do not allow to pass-in a state store
supplier here any more

// additional code to the topology above, could be more prescriptive
than descriptive
// only advanced users would want to code in both parts above; while other
users would only code the topology as above.

table1.materialize("queryableStoreName"); // or..
table1.materialize("queryableStoreName").enableCaching().enableLogging();
// or..
table1.materialize(stateStoreSupplier); // we check type (key-value types,
windowed or not etc) at starting time and add the metrics / logging /
caching / windowing wrapper on top of the store, or..
table1.materialize(stateStoreSupplier).enableCaching().enableLogging(); //
etc..

---------------------------------------

But one caveat of that, as illustrated above, is that you need to have
separate object of the KTable in order to call either "queryHandle" or
"materialize" (whatever the function name is) for the specifications of
materialization options. This can break the concatenation of the topology
construction part of the code, that you cannot simply add one operator
directly after another. So I think this is a trade-off we have to make and
the current approach looks better in this regard.



Guozhang




On Wed, Aug 2, 2017 at 2:07 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> Hi Bill,
>
> totally! So in the original discussion it was mentioned that the overloads
> are nasty when implementing new features. So we wanted to get rid of them.
> But what I felt was that the
> copy & pasted code in the KTableProcessors for maintaining IQ stores was
> as big as a hurdle as the overloads.
>
> With this proposal I try to shift things into the direction of getting IQ
> for free if
> KTableValueGetterSupplier is properly implemented (like getting join for
> free then). Instead of having the code for maintaining IQ stores all the
> places. I realized I can do that while getting rid of the overloads, that
> makes me feel my proposal is superior.
>
> Further I try to optimize by using as few stores as possible to give the
> user what he needs. That should save all sorts of resources while allowing
> faster rebalances.
>
> The target ultimately is to only have KTableSource and the Aggregators
> maintain a Store and provide a ValueGetterSupplier.
>
> Does this makes sense to you?
>
> Best Jan
>
> On 02.08.2017 18:09, Bill Bejeck wrote:
>
>> Hi Jan,
>>
>> Thanks for the effort in putting your thoughts down on paper.
>>
>> Comparing what I see from your proposal and what is presented in KIP-182,
>> one of the main differences is the exclusion of an`Materialized`  instance
>> in the `KTable` methods.
>>
>> Can you go into more detail why this is so and the specific problems is
>> avoids and or solves with this approach?
>>
>> Thanks!
>> Bill
>>
>> On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <damian.guy@gmail.com <mailto:
>> damian.guy@gmail.com>> wrote:
>>
>>     Hi Jan,
>>
>>     Thanks for taking the time to put this together, appreciated. For the
>>     benefit of others would you mind explaining a bit about your
>>     motivation?
>>
>>     Cheers,
>>     Damian
>>
>>     On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Jan.Filipiak@trivago.com
>>     <ma...@trivago.com>> wrote:
>>
>>     > Hi all,
>>     >
>>     > after some further discussions, the best thing to show my Idea
>>     of how it
>>     > should evolve would be a bigger mock/interface description.
>>     > The goal is to reduce the store maintaining processors to only the
>>     > Aggregators + and KTableSource. While having KTableSource optionally
>>     > materialized.
>>     >
>>     > Introducing KTable:copy() will allow users to maintain state
>>     twice if
>>     > they really want to. KStream::join*() wasn't touched. I never
>>     personally
>>     > used that so I didn't feel
>>     > comfortable enough touching it. Currently still making up my
>>     mind. None
>>     > of the suggestions made it querieable so far. Gouzhangs
>>     'Buffered' idea
>>     > seems ideal here.
>>     >
>>     > please have a look. Looking forward for your opinions.
>>     >
>>     > Best Jan
>>     >
>>     >
>>     >
>>     > On 21.06.2017 17 <tel:21.06.2017%2017>:24, Eno Thereska wrote:
>>     > > (cc’ing user-list too)
>>     > >
>>     > > Given that we already have StateStoreSuppliers that are
>>     configurable
>>     > using the fluent-like API, probably it’s worth discussing the other
>>     > examples with joins and serdes first since those have many
>>     overloads and
>>     > are in need of some TLC.
>>     > >
>>     > > So following your example, I guess you’d have something like:
>>     > > .join()
>>     > >     .withKeySerdes(…)
>>     > >     .withValueSerdes(…)
>>     > >     .withJoinType(“outer”)
>>     > >
>>     > > etc?
>>     > >
>>     > > I like the approach since it still remains declarative and
>>     it’d reduce
>>     > the number of overloads by quite a bit.
>>     > >
>>     > > Eno
>>     > >
>>     > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>     <ma...@gmail.com>> wrote:
>>     > >>
>>     > >> Hi,
>>     > >>
>>     > >> I'd like to get a discussion going around some of the API
>>     choices we've
>>     > >> made in the DLS. In particular those that relate to stateful
>>     operations
>>     > >> (though this could expand).
>>     > >> As it stands we lean heavily on overloaded methods in the
>>     API, i.e,
>>     > there
>>     > >> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>     noisy and i
>>     > >> feel it is only going to get worse as we add more optional
>>     params. In
>>     > >> particular we've had some requests to be able to turn caching
>>     off, or
>>     > >> change log configs,  on a per operator basis (note this can
>>     be done now
>>     > if
>>     > >> you pass in a StateStoreSupplier, but this can be a bit
>>     cumbersome).
>>     > >>
>>     > >> So this is a bit of an open question. How can we change the DSL
>>     > overloads
>>     > >> so that it flows, is simple to use and understand, and is easily
>>     > extended
>>     > >> in the future?
>>     > >>
>>     > >> One option would be to use a fluent API approach for
>>     providing the
>>     > optional
>>     > >> params, so something like this:
>>     > >>
>>     > >> groupedStream.count()
>>     > >>    .withStoreName("name")
>>     > >>    .withCachingEnabled(false)
>>     > >>    .withLoggingEnabled(config)
>>     > >>    .table()
>>     > >>
>>     > >>
>>     > >>
>>     > >> Another option would be to provide a Builder to the count
>>     method, so it
>>     > >> would look something like this:
>>     > >> groupedStream.count(new
>>     > >> CountBuilder("storeName").withCachingEnabled(false).build())
>>     > >>
>>     > >> Another option is to say: Hey we don't need this, what are
>>     you on about!
>>     > >>
>>     > >> The above has focussed on state store related overloads, but
>>     the same
>>     > ideas
>>     > >> could  be applied to joins etc, where we presently have many join
>>     > methods
>>     > >> and many overloads.
>>     > >>
>>     > >> Anyway, i look forward to hearing your opinions.
>>     > >>
>>     > >> Thanks,
>>     > >> Damian
>>     >
>>     >
>>
>>
>>
>


-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Bill,

totally! So in the original discussion it was mentioned that the 
overloads are nasty when implementing new features. So we wanted to get 
rid of them. But what I felt was that the
copy & pasted code in the KTableProcessors for maintaining IQ stores was 
as big as a hurdle as the overloads.

With this proposal I try to shift things into the direction of getting 
IQ for free if
KTableValueGetterSupplier is properly implemented (like getting join for 
free then). Instead of having the code for maintaining IQ stores all the 
places. I realized I can do that while getting rid of the overloads, 
that makes me feel my proposal is superior.

Further I try to optimize by using as few stores as possible to give the 
user what he needs. That should save all sorts of resources while 
allowing faster rebalances.

The target ultimately is to only have KTableSource and the Aggregators 
maintain a Store and provide a ValueGetterSupplier.

Does this makes sense to you?

Best Jan

On 02.08.2017 18:09, Bill Bejeck wrote:
> Hi Jan,
>
> Thanks for the effort in putting your thoughts down on paper.
>
> Comparing what I see from your proposal and what is presented in 
> KIP-182, one of the main differences is the exclusion of 
> an`Materialized`  instance in the `KTable` methods.
>
> Can you go into more detail why this is so and the specific problems 
> is avoids and or solves with this approach?
>
> Thanks!
> Bill
>
> On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <damian.guy@gmail.com 
> <ma...@gmail.com>> wrote:
>
>     Hi Jan,
>
>     Thanks for taking the time to put this together, appreciated. For the
>     benefit of others would you mind explaining a bit about your
>     motivation?
>
>     Cheers,
>     Damian
>
>     On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Jan.Filipiak@trivago.com
>     <ma...@trivago.com>> wrote:
>
>     > Hi all,
>     >
>     > after some further discussions, the best thing to show my Idea
>     of how it
>     > should evolve would be a bigger mock/interface description.
>     > The goal is to reduce the store maintaining processors to only the
>     > Aggregators + and KTableSource. While having KTableSource optionally
>     > materialized.
>     >
>     > Introducing KTable:copy() will allow users to maintain state
>     twice if
>     > they really want to. KStream::join*() wasn't touched. I never
>     personally
>     > used that so I didn't feel
>     > comfortable enough touching it. Currently still making up my
>     mind. None
>     > of the suggestions made it querieable so far. Gouzhangs
>     'Buffered' idea
>     > seems ideal here.
>     >
>     > please have a look. Looking forward for your opinions.
>     >
>     > Best Jan
>     >
>     >
>     >
>     > On 21.06.2017 17 <tel:21.06.2017%2017>:24, Eno Thereska wrote:
>     > > (cc’ing user-list too)
>     > >
>     > > Given that we already have StateStoreSuppliers that are
>     configurable
>     > using the fluent-like API, probably it’s worth discussing the other
>     > examples with joins and serdes first since those have many
>     overloads and
>     > are in need of some TLC.
>     > >
>     > > So following your example, I guess you’d have something like:
>     > > .join()
>     > >     .withKeySerdes(…)
>     > >     .withValueSerdes(…)
>     > >     .withJoinType(“outer”)
>     > >
>     > > etc?
>     > >
>     > > I like the approach since it still remains declarative and
>     it’d reduce
>     > the number of overloads by quite a bit.
>     > >
>     > > Eno
>     > >
>     > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <damian.guy@gmail.com
>     <ma...@gmail.com>> wrote:
>     > >>
>     > >> Hi,
>     > >>
>     > >> I'd like to get a discussion going around some of the API
>     choices we've
>     > >> made in the DLS. In particular those that relate to stateful
>     operations
>     > >> (though this could expand).
>     > >> As it stands we lean heavily on overloaded methods in the
>     API, i.e,
>     > there
>     > >> are 9 overloads for KGroupedStream.count(..)! It is becoming
>     noisy and i
>     > >> feel it is only going to get worse as we add more optional
>     params. In
>     > >> particular we've had some requests to be able to turn caching
>     off, or
>     > >> change log configs,  on a per operator basis (note this can
>     be done now
>     > if
>     > >> you pass in a StateStoreSupplier, but this can be a bit
>     cumbersome).
>     > >>
>     > >> So this is a bit of an open question. How can we change the DSL
>     > overloads
>     > >> so that it flows, is simple to use and understand, and is easily
>     > extended
>     > >> in the future?
>     > >>
>     > >> One option would be to use a fluent API approach for
>     providing the
>     > optional
>     > >> params, so something like this:
>     > >>
>     > >> groupedStream.count()
>     > >>    .withStoreName("name")
>     > >>    .withCachingEnabled(false)
>     > >>    .withLoggingEnabled(config)
>     > >>    .table()
>     > >>
>     > >>
>     > >>
>     > >> Another option would be to provide a Builder to the count
>     method, so it
>     > >> would look something like this:
>     > >> groupedStream.count(new
>     > >> CountBuilder("storeName").withCachingEnabled(false).build())
>     > >>
>     > >> Another option is to say: Hey we don't need this, what are
>     you on about!
>     > >>
>     > >> The above has focussed on state store related overloads, but
>     the same
>     > ideas
>     > >> could  be applied to joins etc, where we presently have many join
>     > methods
>     > >> and many overloads.
>     > >>
>     > >> Anyway, i look forward to hearing your opinions.
>     > >>
>     > >> Thanks,
>     > >> Damian
>     >
>     >
>
>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Bill Bejeck <bi...@confluent.io>.
Hi Jan,

Thanks for the effort in putting your thoughts down on paper.

Comparing what I see from your proposal and what is presented in KIP-182,
one of the main differences is the exclusion of an`Materialized`  instance
in the `KTable` methods.

Can you go into more detail why this is so and the specific problems is
avoids and or solves with this approach?

Thanks!
Bill

On Wed, Aug 2, 2017 at 4:19 AM, Damian Guy <da...@gmail.com> wrote:

> Hi Jan,
>
> Thanks for taking the time to put this together, appreciated. For the
> benefit of others would you mind explaining a bit about your motivation?
>
> Cheers,
> Damian
>
> On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Ja...@trivago.com> wrote:
>
> > Hi all,
> >
> > after some further discussions, the best thing to show my Idea of how it
> > should evolve would be a bigger mock/interface description.
> > The goal is to reduce the store maintaining processors to only the
> > Aggregators + and KTableSource. While having KTableSource optionally
> > materialized.
> >
> > Introducing KTable:copy() will allow users to maintain state twice if
> > they really want to. KStream::join*() wasn't touched. I never personally
> > used that so I didn't feel
> > comfortable enough touching it. Currently still making up my mind. None
> > of the suggestions made it querieable so far. Gouzhangs 'Buffered' idea
> > seems ideal here.
> >
> > please have a look. Looking forward for your opinions.
> >
> > Best Jan
> >
> >
> >
> > On 21.06.2017 17:24, Eno Thereska wrote:
> > > (cc’ing user-list too)
> > >
> > > Given that we already have StateStoreSuppliers that are configurable
> > using the fluent-like API, probably it’s worth discussing the other
> > examples with joins and serdes first since those have many overloads and
> > are in need of some TLC.
> > >
> > > So following your example, I guess you’d have something like:
> > > .join()
> > >     .withKeySerdes(…)
> > >     .withValueSerdes(…)
> > >     .withJoinType(“outer”)
> > >
> > > etc?
> > >
> > > I like the approach since it still remains declarative and it’d reduce
> > the number of overloads by quite a bit.
> > >
> > > Eno
> > >
> > >> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> > >>
> > >> Hi,
> > >>
> > >> I'd like to get a discussion going around some of the API choices
> we've
> > >> made in the DLS. In particular those that relate to stateful
> operations
> > >> (though this could expand).
> > >> As it stands we lean heavily on overloaded methods in the API, i.e,
> > there
> > >> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> and i
> > >> feel it is only going to get worse as we add more optional params. In
> > >> particular we've had some requests to be able to turn caching off, or
> > >> change log configs,  on a per operator basis (note this can be done
> now
> > if
> > >> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> > >>
> > >> So this is a bit of an open question. How can we change the DSL
> > overloads
> > >> so that it flows, is simple to use and understand, and is easily
> > extended
> > >> in the future?
> > >>
> > >> One option would be to use a fluent API approach for providing the
> > optional
> > >> params, so something like this:
> > >>
> > >> groupedStream.count()
> > >>    .withStoreName("name")
> > >>    .withCachingEnabled(false)
> > >>    .withLoggingEnabled(config)
> > >>    .table()
> > >>
> > >>
> > >>
> > >> Another option would be to provide a Builder to the count method, so
> it
> > >> would look something like this:
> > >> groupedStream.count(new
> > >> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>
> > >> Another option is to say: Hey we don't need this, what are you on
> about!
> > >>
> > >> The above has focussed on state store related overloads, but the same
> > ideas
> > >> could  be applied to joins etc, where we presently have many join
> > methods
> > >> and many overloads.
> > >>
> > >> Anyway, i look forward to hearing your opinions.
> > >>
> > >> Thanks,
> > >> Damian
> >
> >
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Jan,

Thanks for taking the time to put this together, appreciated. For the
benefit of others would you mind explaining a bit about your motivation?

Cheers,
Damian

On Wed, 2 Aug 2017 at 01:40 Jan Filipiak <Ja...@trivago.com> wrote:

> Hi all,
>
> after some further discussions, the best thing to show my Idea of how it
> should evolve would be a bigger mock/interface description.
> The goal is to reduce the store maintaining processors to only the
> Aggregators + and KTableSource. While having KTableSource optionally
> materialized.
>
> Introducing KTable:copy() will allow users to maintain state twice if
> they really want to. KStream::join*() wasn't touched. I never personally
> used that so I didn't feel
> comfortable enough touching it. Currently still making up my mind. None
> of the suggestions made it querieable so far. Gouzhangs 'Buffered' idea
> seems ideal here.
>
> please have a look. Looking forward for your opinions.
>
> Best Jan
>
>
>
> On 21.06.2017 17:24, Eno Thereska wrote:
> > (cc’ing user-list too)
> >
> > Given that we already have StateStoreSuppliers that are configurable
> using the fluent-like API, probably it’s worth discussing the other
> examples with joins and serdes first since those have many overloads and
> are in need of some TLC.
> >
> > So following your example, I guess you’d have something like:
> > .join()
> >     .withKeySerdes(…)
> >     .withValueSerdes(…)
> >     .withJoinType(“outer”)
> >
> > etc?
> >
> > I like the approach since it still remains declarative and it’d reduce
> the number of overloads by quite a bit.
> >
> > Eno
> >
> >> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> >>
> >> Hi,
> >>
> >> I'd like to get a discussion going around some of the API choices we've
> >> made in the DLS. In particular those that relate to stateful operations
> >> (though this could expand).
> >> As it stands we lean heavily on overloaded methods in the API, i.e,
> there
> >> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> >> feel it is only going to get worse as we add more optional params. In
> >> particular we've had some requests to be able to turn caching off, or
> >> change log configs,  on a per operator basis (note this can be done now
> if
> >> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>
> >> So this is a bit of an open question. How can we change the DSL
> overloads
> >> so that it flows, is simple to use and understand, and is easily
> extended
> >> in the future?
> >>
> >> One option would be to use a fluent API approach for providing the
> optional
> >> params, so something like this:
> >>
> >> groupedStream.count()
> >>    .withStoreName("name")
> >>    .withCachingEnabled(false)
> >>    .withLoggingEnabled(config)
> >>    .table()
> >>
> >>
> >>
> >> Another option would be to provide a Builder to the count method, so it
> >> would look something like this:
> >> groupedStream.count(new
> >> CountBuilder("storeName").withCachingEnabled(false).build())
> >>
> >> Another option is to say: Hey we don't need this, what are you on about!
> >>
> >> The above has focussed on state store related overloads, but the same
> ideas
> >> could  be applied to joins etc, where we presently have many join
> methods
> >> and many overloads.
> >>
> >> Anyway, i look forward to hearing your opinions.
> >>
> >> Thanks,
> >> Damian
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi all,

after some further discussions, the best thing to show my Idea of how it 
should evolve would be a bigger mock/interface description.
The goal is to reduce the store maintaining processors to only the 
Aggregators + and KTableSource. While having KTableSource optionally 
materialized.

Introducing KTable:copy() will allow users to maintain state twice if 
they really want to. KStream::join*() wasn't touched. I never personally 
used that so I didn't feel
comfortable enough touching it. Currently still making up my mind. None 
of the suggestions made it querieable so far. Gouzhangs 'Buffered' idea 
seems ideal here.

please have a look. Looking forward for your opinions.

Best Jan



On 21.06.2017 17:24, Eno Thereska wrote:
> (cc’ing user-list too)
>
> Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC.
>
> So following your example, I guess you’d have something like:
> .join()
>     .withKeySerdes(…)
>     .withValueSerdes(…)
>     .withJoinType(“outer”)
>
> etc?
>
> I like the approach since it still remains declarative and it’d reduce the number of overloads by quite a bit.
>
> Eno
>
>> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>
>> Hi,
>>
>> I'd like to get a discussion going around some of the API choices we've
>> made in the DLS. In particular those that relate to stateful operations
>> (though this could expand).
>> As it stands we lean heavily on overloaded methods in the API, i.e, there
>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
>> feel it is only going to get worse as we add more optional params. In
>> particular we've had some requests to be able to turn caching off, or
>> change log configs,  on a per operator basis (note this can be done now if
>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>
>> So this is a bit of an open question. How can we change the DSL overloads
>> so that it flows, is simple to use and understand, and is easily extended
>> in the future?
>>
>> One option would be to use a fluent API approach for providing the optional
>> params, so something like this:
>>
>> groupedStream.count()
>>    .withStoreName("name")
>>    .withCachingEnabled(false)
>>    .withLoggingEnabled(config)
>>    .table()
>>
>>
>>
>> Another option would be to provide a Builder to the count method, so it
>> would look something like this:
>> groupedStream.count(new
>> CountBuilder("storeName").withCachingEnabled(false).build())
>>
>> Another option is to say: Hey we don't need this, what are you on about!
>>
>> The above has focussed on state store related overloads, but the same ideas
>> could  be applied to joins etc, where we presently have many join methods
>> and many overloads.
>>
>> Anyway, i look forward to hearing your opinions.
>>
>> Thanks,
>> Damian


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
(cc’ing user-list too)

Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC.

So following your example, I guess you’d have something like:
.join()
   .withKeySerdes(…)
   .withValueSerdes(…)
   .withJoinType(“outer”)

etc?

I like the approach since it still remains declarative and it’d reduce the number of overloads by quite a bit.

Eno

> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> 
> Hi,
> 
> I'd like to get a discussion going around some of the API choices we've
> made in the DLS. In particular those that relate to stateful operations
> (though this could expand).
> As it stands we lean heavily on overloaded methods in the API, i.e, there
> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> feel it is only going to get worse as we add more optional params. In
> particular we've had some requests to be able to turn caching off, or
> change log configs,  on a per operator basis (note this can be done now if
> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> 
> So this is a bit of an open question. How can we change the DSL overloads
> so that it flows, is simple to use and understand, and is easily extended
> in the future?
> 
> One option would be to use a fluent API approach for providing the optional
> params, so something like this:
> 
> groupedStream.count()
>   .withStoreName("name")
>   .withCachingEnabled(false)
>   .withLoggingEnabled(config)
>   .table()
> 
> 
> 
> Another option would be to provide a Builder to the count method, so it
> would look something like this:
> groupedStream.count(new
> CountBuilder("storeName").withCachingEnabled(false).build())
> 
> Another option is to say: Hey we don't need this, what are you on about!
> 
> The above has focussed on state store related overloads, but the same ideas
> could  be applied to joins etc, where we presently have many join methods
> and many overloads.
> 
> Anyway, i look forward to hearing your opinions.
> 
> Thanks,
> Damian


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Eno Thereska <en...@gmail.com>.
(cc’ing user-list too)

Given that we already have StateStoreSuppliers that are configurable using the fluent-like API, probably it’s worth discussing the other examples with joins and serdes first since those have many overloads and are in need of some TLC.

So following your example, I guess you’d have something like:
.join()
   .withKeySerdes(…)
   .withValueSerdes(…)
   .withJoinType(“outer”)

etc?

I like the approach since it still remains declarative and it’d reduce the number of overloads by quite a bit.

Eno

> On Jun 21, 2017, at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
> 
> Hi,
> 
> I'd like to get a discussion going around some of the API choices we've
> made in the DLS. In particular those that relate to stateful operations
> (though this could expand).
> As it stands we lean heavily on overloaded methods in the API, i.e, there
> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> feel it is only going to get worse as we add more optional params. In
> particular we've had some requests to be able to turn caching off, or
> change log configs,  on a per operator basis (note this can be done now if
> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> 
> So this is a bit of an open question. How can we change the DSL overloads
> so that it flows, is simple to use and understand, and is easily extended
> in the future?
> 
> One option would be to use a fluent API approach for providing the optional
> params, so something like this:
> 
> groupedStream.count()
>   .withStoreName("name")
>   .withCachingEnabled(false)
>   .withLoggingEnabled(config)
>   .table()
> 
> 
> 
> Another option would be to provide a Builder to the count method, so it
> would look something like this:
> groupedStream.count(new
> CountBuilder("storeName").withCachingEnabled(false).build())
> 
> Another option is to say: Hey we don't need this, what are you on about!
> 
> The above has focussed on state store related overloads, but the same ideas
> could  be applied to joins etc, where we presently have many join methods
> and many overloads.
> 
> Anyway, i look forward to hearing your opinions.
> 
> Thanks,
> Damian


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
I've updated the experimental code with a couple of ways of doing joins.
One following the fluent approach and one following the builder approach.
The 2 examples can be found here:
https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java#L714

The code looks like:

@Test
public void shouldBeFluentIsh() throws Exception {
    final KStream<String, String> one = null;
    final KStream<String, String> two = null;
    final Serde<String> serde = null;
    final ValueJoiner<String, String, String> vj = null;

    // inner join
    one.join2(two, vj, JoinWindows.of(10))
            .withKeySerde(serde)
            .withThisValueSerde(serde)
            .withOtherValueSerde(serde)
            .stream();

    // left join
    one.join2(two, vj, JoinWindows.of(10))
            .withJoinType(JoinType.LEFT)
            .stream();
}

@Test
public void shouldUseBuilder() throws Exception {
    final KStream<String, String> one = null;
    final KStream<String, String> two = null;
    final Serde<String> serde = null;
    final ValueJoiner<String, String, String> vj = null;

    //inner
    one.join(Joins.streamStreamJoin(two, vj, JoinWindows.of(10)).build());

    //left
    one.join(Joins.streamStreamJoin(two, vj,
JoinWindows.of(10)).withJoinType(JoinType.LEFT).build());
}


I'm not going to say which way i'm leaning, yet!

Thanks,
Damian

On Thu, 29 Jun 2017 at 11:47 Damian Guy <da...@gmail.com> wrote:

>
>> However, I don't understand your argument about putting aggregate()
>> after the withXX() -- all the calls to withXX() set optional parameters
>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>> be quite confusion for developers.
>>
>>
> I see what you are saying, but the grouped stream is effectively a no-op
> until you call one of the aggregate/count/reduce etc functions. So the
> optional params are ones that are applicable to any of the operations you
> can perform on this grouped stream. Then the final
> count()/reduce()/aggregate() call has any of the params that are
> required/specific to that function.
>
>
>>
>> -Matthias
>>
>> On 6/28/17 2:55 AM, Damian Guy wrote:
>> >> I also think that mixing optional parameters with configs is a bad
>> idea.
>> >> Have not proposal for this atm but just wanted to mention it. Hope to
>> >> find some time to come up with something.
>> >>
>> >>
>> > Yes, i don't like the mix of config either. But the only real config
>> here
>> > is the logging config - which we don't really need as it can already be
>> > done via a custom StateStoreSupplier.
>> >
>> >
>> >> What I don't like in the current proposal is the
>> >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
>> >> and .groupByKey() seems better. For clarity, we could rename to
>> >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>> >> some better names).
>> >>
>> >>
>> > it could be groupByKey(), groupBy() or something different bt
>> >
>> >
>> >
>> >> The proposed pattern "chains" grouping and aggregation too close
>> >> together. I would rather separate both more than less, ie, do into the
>> >> opposite direction.
>> >>
>> >> I am also wondering, if we could so something more "fluent". The
>> initial
>> >> proposal was like:
>> >>
>> >>>> groupedStream.count()
>> >>>>    .withStoreName("name")
>> >>>>    .withCachingEnabled(false)
>> >>>>    .withLoggingEnabled(config)
>> >>>>    .table()
>> >>
>> >> The .table() statement in the end was kinda alien.
>> >>
>> >
>> > I agree, but then all of the withXXX methods need to be on KTable which
>> is
>> > worse in my opinion. You also need something that is going to "build"
>> the
>> > internal processors and add them to the topology.
>> >
>> >
>> >> The current proposal put the count() into the end -- ie, the optional
>> >> parameter for count() have to specified on the .grouped() call -- this
>> >> does not seems to be the best way either.
>> >>
>> >>
>> > I actually prefer this method as you are building a grouped stream that
>> you
>> > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
>> etc
>> > seems natural to me.
>> >
>> >
>> >> I did not think this through in detail, but can't we just do the
>> initial
>> >> proposal with the .table() ?
>> >>
>> >> groupedStream.count().withStoreName("name").mapValues(...)
>> >>
>> >> Each .withXXX(...) return the current KTable and all the .withXXX() are
>> >> just added to the KTable interface. Or do I miss anything why this
>> wont'
>> >> work or any obvious disadvantage?
>> >>
>> >>
>> >>
>> > See above.
>> >
>> >
>> >>
>> >> -Matthias
>> >>
>> >> On 6/22/17 4:06 AM, Damian Guy wrote:
>> >>> Thanks everyone. My latest attempt is below. It builds on the fluent
>> >>> approach, but i think it is slightly nicer.
>> >>> I agree with some of what Eno said about mixing configy stuff in the
>> DSL,
>> >>> but i think that enabling caching and enabling logging are things that
>> >>> aren't actually config. I'd probably not add withLogConfig(...) (even
>> >>> though it is below) as this is actually config and we already have a
>> way
>> >> of
>> >>> doing that, via the StateStoreSupplier. Arguably we could use the
>> >>> StateStoreSupplier for disabling caching etc, but as it stands that
>> is a
>> >>> bit of a tedious process for someone that just wants to use the
>> default
>> >>> storage engine, but not have caching enabled.
>> >>>
>> >>> There is also an orthogonal concern that Guozhang alluded to.... If
>> you
>> >>> want to plug in a custom storage engine and you want it to be logged
>> etc,
>> >>> you would currently need to implement that yourself. Ideally we can
>> >> provide
>> >>> a way where we will wrap the custom store with logging, metrics, etc.
>> I
>> >>> need to think about where this fits, it is probably more appropriate
>> on
>> >> the
>> >>> Stores API.
>> >>>
>> >>> final KeyValueMapper<String, String, Long> keyMapper = null;
>> >>> // count with mapped key
>> >>> final KTable<Long, Long> count = stream.grouped()
>> >>>         .withKeyMapper(keyMapper)
>> >>>         .withKeySerde(Serdes.Long())
>> >>>         .withValueSerde(Serdes.String())
>> >>>         .withQueryableName("my-store")
>> >>>         .count();
>> >>>
>> >>> // windowed count
>> >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
>> >>>         .withQueryableName("my-window-store")
>> >>>         .windowed(TimeWindows.of(10L).until(10))
>> >>>         .count();
>> >>>
>> >>> // windowed reduce
>> >>> final Reducer<String> windowedReducer = null;
>> >>> final KTable<Windowed<String>, String> windowedReduce =
>> stream.grouped()
>> >>>         .withQueryableName("my-window-store")
>> >>>         .windowed(TimeWindows.of(10L).until(10))
>> >>>         .reduce(windowedReducer);
>> >>>
>> >>> final Aggregator<String, String, Long> aggregator = null;
>> >>> final Initializer<Long> init = null;
>> >>>
>> >>> // aggregate
>> >>> final KTable<String, Long> aggregate = stream.grouped()
>> >>>         .withQueryableName("my-aggregate-store")
>> >>>         .aggregate(aggregator, init, Serdes.Long());
>> >>>
>> >>> final StateStoreSupplier<KeyValueStore<String, Long>>
>> stateStoreSupplier
>> >> = null;
>> >>>
>> >>> // aggregate with custom store
>> >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>> >>>         .withStateStoreSupplier(stateStoreSupplier)
>> >>>         .aggregate(aggregator, init);
>> >>>
>> >>> // disable caching
>> >>> stream.grouped()
>> >>>         .withQueryableName("name")
>> >>>         .withCachingEnabled(false)
>> >>>         .count();
>> >>>
>> >>> // disable logging
>> >>> stream.grouped()
>> >>>         .withQueryableName("q")
>> >>>         .withLoggingEnabled(false)
>> >>>         .count();
>> >>>
>> >>> // override log config
>> >>> final Reducer<String> reducer = null;
>> >>> stream.grouped()
>> >>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
>> >>>         .reduce(reducer);
>> >>>
>> >>>
>> >>> If anyone wants to play around with this you can find the code here:
>> >>> https://github.com/dguy/kafka/tree/dsl-experiment
>> >>>
>> >>> Note: It won't actually work as most of the methods just return null.
>> >>>
>> >>> Thanks,
>> >>> Damian
>> >>>
>> >>>
>> >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
>> >>>
>> >>>> Thanks Damian. I think both options have pros and cons. And both are
>> >> better
>> >>>> than overload abuse.
>> >>>>
>> >>>> The fluent API approach reads better, no mention of builder or build
>> >>>> anywhere. The main downside is that the method signatures are a
>> little
>> >> less
>> >>>> clear. By reading the method signature, one doesn't necessarily knows
>> >> what
>> >>>> it returns. Also, one needs to figure out the special method
>> (`table()`
>> >> in
>> >>>> this case) that gives you what you actually care about (`KTable` in
>> this
>> >>>> case). Not major issues, but worth mentioning while doing the
>> >> comparison.
>> >>>>
>> >>>> The builder approach avoids the issues mentioned above, but it
>> doesn't
>> >> read
>> >>>> as well.
>> >>>>
>> >>>> Ismael
>> >>>>
>> >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> Hi,
>> >>>>>
>> >>>>> I'd like to get a discussion going around some of the API choices
>> we've
>> >>>>> made in the DLS. In particular those that relate to stateful
>> operations
>> >>>>> (though this could expand).
>> >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>> >> there
>> >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
>> and
>> >> i
>> >>>>> feel it is only going to get worse as we add more optional params.
>> In
>> >>>>> particular we've had some requests to be able to turn caching off,
>> or
>> >>>>> change log configs,  on a per operator basis (note this can be done
>> now
>> >>>> if
>> >>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>> >>>>>
>> >>>>> So this is a bit of an open question. How can we change the DSL
>> >> overloads
>> >>>>> so that it flows, is simple to use and understand, and is easily
>> >> extended
>> >>>>> in the future?
>> >>>>>
>> >>>>> One option would be to use a fluent API approach for providing the
>> >>>> optional
>> >>>>> params, so something like this:
>> >>>>>
>> >>>>> groupedStream.count()
>> >>>>>    .withStoreName("name")
>> >>>>>    .withCachingEnabled(false)
>> >>>>>    .withLoggingEnabled(config)
>> >>>>>    .table()
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> Another option would be to provide a Builder to the count method,
>> so it
>> >>>>> would look something like this:
>> >>>>> groupedStream.count(new
>> >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>> >>>>>
>> >>>>> Another option is to say: Hey we don't need this, what are you on
>> >> about!
>> >>>>>
>> >>>>> The above has focussed on state store related overloads, but the
>> same
>> >>>> ideas
>> >>>>> could  be applied to joins etc, where we presently have many join
>> >> methods
>> >>>>> and many overloads.
>> >>>>>
>> >>>>> Anyway, i look forward to hearing your opinions.
>> >>>>>
>> >>>>> Thanks,
>> >>>>> Damian
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >>
>> >
>>
>>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
I've updated the experimental code with a couple of ways of doing joins.
One following the fluent approach and one following the builder approach.
The 2 examples can be found here:
https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java#L714

The code looks like:

@Test
public void shouldBeFluentIsh() throws Exception {
    final KStream<String, String> one = null;
    final KStream<String, String> two = null;
    final Serde<String> serde = null;
    final ValueJoiner<String, String, String> vj = null;

    // inner join
    one.join2(two, vj, JoinWindows.of(10))
            .withKeySerde(serde)
            .withThisValueSerde(serde)
            .withOtherValueSerde(serde)
            .stream();

    // left join
    one.join2(two, vj, JoinWindows.of(10))
            .withJoinType(JoinType.LEFT)
            .stream();
}

@Test
public void shouldUseBuilder() throws Exception {
    final KStream<String, String> one = null;
    final KStream<String, String> two = null;
    final Serde<String> serde = null;
    final ValueJoiner<String, String, String> vj = null;

    //inner
    one.join(Joins.streamStreamJoin(two, vj, JoinWindows.of(10)).build());

    //left
    one.join(Joins.streamStreamJoin(two, vj,
JoinWindows.of(10)).withJoinType(JoinType.LEFT).build());
}


I'm not going to say which way i'm leaning, yet!

Thanks,
Damian

On Thu, 29 Jun 2017 at 11:47 Damian Guy <da...@gmail.com> wrote:

>
>> However, I don't understand your argument about putting aggregate()
>> after the withXX() -- all the calls to withXX() set optional parameters
>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>> be quite confusion for developers.
>>
>>
> I see what you are saying, but the grouped stream is effectively a no-op
> until you call one of the aggregate/count/reduce etc functions. So the
> optional params are ones that are applicable to any of the operations you
> can perform on this grouped stream. Then the final
> count()/reduce()/aggregate() call has any of the params that are
> required/specific to that function.
>
>
>>
>> -Matthias
>>
>> On 6/28/17 2:55 AM, Damian Guy wrote:
>> >> I also think that mixing optional parameters with configs is a bad
>> idea.
>> >> Have not proposal for this atm but just wanted to mention it. Hope to
>> >> find some time to come up with something.
>> >>
>> >>
>> > Yes, i don't like the mix of config either. But the only real config
>> here
>> > is the logging config - which we don't really need as it can already be
>> > done via a custom StateStoreSupplier.
>> >
>> >
>> >> What I don't like in the current proposal is the
>> >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
>> >> and .groupByKey() seems better. For clarity, we could rename to
>> >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>> >> some better names).
>> >>
>> >>
>> > it could be groupByKey(), groupBy() or something different bt
>> >
>> >
>> >
>> >> The proposed pattern "chains" grouping and aggregation too close
>> >> together. I would rather separate both more than less, ie, do into the
>> >> opposite direction.
>> >>
>> >> I am also wondering, if we could so something more "fluent". The
>> initial
>> >> proposal was like:
>> >>
>> >>>> groupedStream.count()
>> >>>>    .withStoreName("name")
>> >>>>    .withCachingEnabled(false)
>> >>>>    .withLoggingEnabled(config)
>> >>>>    .table()
>> >>
>> >> The .table() statement in the end was kinda alien.
>> >>
>> >
>> > I agree, but then all of the withXXX methods need to be on KTable which
>> is
>> > worse in my opinion. You also need something that is going to "build"
>> the
>> > internal processors and add them to the topology.
>> >
>> >
>> >> The current proposal put the count() into the end -- ie, the optional
>> >> parameter for count() have to specified on the .grouped() call -- this
>> >> does not seems to be the best way either.
>> >>
>> >>
>> > I actually prefer this method as you are building a grouped stream that
>> you
>> > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
>> etc
>> > seems natural to me.
>> >
>> >
>> >> I did not think this through in detail, but can't we just do the
>> initial
>> >> proposal with the .table() ?
>> >>
>> >> groupedStream.count().withStoreName("name").mapValues(...)
>> >>
>> >> Each .withXXX(...) return the current KTable and all the .withXXX() are
>> >> just added to the KTable interface. Or do I miss anything why this
>> wont'
>> >> work or any obvious disadvantage?
>> >>
>> >>
>> >>
>> > See above.
>> >
>> >
>> >>
>> >> -Matthias
>> >>
>> >> On 6/22/17 4:06 AM, Damian Guy wrote:
>> >>> Thanks everyone. My latest attempt is below. It builds on the fluent
>> >>> approach, but i think it is slightly nicer.
>> >>> I agree with some of what Eno said about mixing configy stuff in the
>> DSL,
>> >>> but i think that enabling caching and enabling logging are things that
>> >>> aren't actually config. I'd probably not add withLogConfig(...) (even
>> >>> though it is below) as this is actually config and we already have a
>> way
>> >> of
>> >>> doing that, via the StateStoreSupplier. Arguably we could use the
>> >>> StateStoreSupplier for disabling caching etc, but as it stands that
>> is a
>> >>> bit of a tedious process for someone that just wants to use the
>> default
>> >>> storage engine, but not have caching enabled.
>> >>>
>> >>> There is also an orthogonal concern that Guozhang alluded to.... If
>> you
>> >>> want to plug in a custom storage engine and you want it to be logged
>> etc,
>> >>> you would currently need to implement that yourself. Ideally we can
>> >> provide
>> >>> a way where we will wrap the custom store with logging, metrics, etc.
>> I
>> >>> need to think about where this fits, it is probably more appropriate
>> on
>> >> the
>> >>> Stores API.
>> >>>
>> >>> final KeyValueMapper<String, String, Long> keyMapper = null;
>> >>> // count with mapped key
>> >>> final KTable<Long, Long> count = stream.grouped()
>> >>>         .withKeyMapper(keyMapper)
>> >>>         .withKeySerde(Serdes.Long())
>> >>>         .withValueSerde(Serdes.String())
>> >>>         .withQueryableName("my-store")
>> >>>         .count();
>> >>>
>> >>> // windowed count
>> >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
>> >>>         .withQueryableName("my-window-store")
>> >>>         .windowed(TimeWindows.of(10L).until(10))
>> >>>         .count();
>> >>>
>> >>> // windowed reduce
>> >>> final Reducer<String> windowedReducer = null;
>> >>> final KTable<Windowed<String>, String> windowedReduce =
>> stream.grouped()
>> >>>         .withQueryableName("my-window-store")
>> >>>         .windowed(TimeWindows.of(10L).until(10))
>> >>>         .reduce(windowedReducer);
>> >>>
>> >>> final Aggregator<String, String, Long> aggregator = null;
>> >>> final Initializer<Long> init = null;
>> >>>
>> >>> // aggregate
>> >>> final KTable<String, Long> aggregate = stream.grouped()
>> >>>         .withQueryableName("my-aggregate-store")
>> >>>         .aggregate(aggregator, init, Serdes.Long());
>> >>>
>> >>> final StateStoreSupplier<KeyValueStore<String, Long>>
>> stateStoreSupplier
>> >> = null;
>> >>>
>> >>> // aggregate with custom store
>> >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>> >>>         .withStateStoreSupplier(stateStoreSupplier)
>> >>>         .aggregate(aggregator, init);
>> >>>
>> >>> // disable caching
>> >>> stream.grouped()
>> >>>         .withQueryableName("name")
>> >>>         .withCachingEnabled(false)
>> >>>         .count();
>> >>>
>> >>> // disable logging
>> >>> stream.grouped()
>> >>>         .withQueryableName("q")
>> >>>         .withLoggingEnabled(false)
>> >>>         .count();
>> >>>
>> >>> // override log config
>> >>> final Reducer<String> reducer = null;
>> >>> stream.grouped()
>> >>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
>> >>>         .reduce(reducer);
>> >>>
>> >>>
>> >>> If anyone wants to play around with this you can find the code here:
>> >>> https://github.com/dguy/kafka/tree/dsl-experiment
>> >>>
>> >>> Note: It won't actually work as most of the methods just return null.
>> >>>
>> >>> Thanks,
>> >>> Damian
>> >>>
>> >>>
>> >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
>> >>>
>> >>>> Thanks Damian. I think both options have pros and cons. And both are
>> >> better
>> >>>> than overload abuse.
>> >>>>
>> >>>> The fluent API approach reads better, no mention of builder or build
>> >>>> anywhere. The main downside is that the method signatures are a
>> little
>> >> less
>> >>>> clear. By reading the method signature, one doesn't necessarily knows
>> >> what
>> >>>> it returns. Also, one needs to figure out the special method
>> (`table()`
>> >> in
>> >>>> this case) that gives you what you actually care about (`KTable` in
>> this
>> >>>> case). Not major issues, but worth mentioning while doing the
>> >> comparison.
>> >>>>
>> >>>> The builder approach avoids the issues mentioned above, but it
>> doesn't
>> >> read
>> >>>> as well.
>> >>>>
>> >>>> Ismael
>> >>>>
>> >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
>> >> wrote:
>> >>>>
>> >>>>> Hi,
>> >>>>>
>> >>>>> I'd like to get a discussion going around some of the API choices
>> we've
>> >>>>> made in the DLS. In particular those that relate to stateful
>> operations
>> >>>>> (though this could expand).
>> >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>> >> there
>> >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
>> and
>> >> i
>> >>>>> feel it is only going to get worse as we add more optional params.
>> In
>> >>>>> particular we've had some requests to be able to turn caching off,
>> or
>> >>>>> change log configs,  on a per operator basis (note this can be done
>> now
>> >>>> if
>> >>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>> >>>>>
>> >>>>> So this is a bit of an open question. How can we change the DSL
>> >> overloads
>> >>>>> so that it flows, is simple to use and understand, and is easily
>> >> extended
>> >>>>> in the future?
>> >>>>>
>> >>>>> One option would be to use a fluent API approach for providing the
>> >>>> optional
>> >>>>> params, so something like this:
>> >>>>>
>> >>>>> groupedStream.count()
>> >>>>>    .withStoreName("name")
>> >>>>>    .withCachingEnabled(false)
>> >>>>>    .withLoggingEnabled(config)
>> >>>>>    .table()
>> >>>>>
>> >>>>>
>> >>>>>
>> >>>>> Another option would be to provide a Builder to the count method,
>> so it
>> >>>>> would look something like this:
>> >>>>> groupedStream.count(new
>> >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>> >>>>>
>> >>>>> Another option is to say: Hey we don't need this, what are you on
>> >> about!
>> >>>>>
>> >>>>> The above has focussed on state store related overloads, but the
>> same
>> >>>> ideas
>> >>>>> could  be applied to joins etc, where we presently have many join
>> >> methods
>> >>>>> and many overloads.
>> >>>>>
>> >>>>> Anyway, i look forward to hearing your opinions.
>> >>>>>
>> >>>>> Thanks,
>> >>>>> Damian
>> >>>>>
>> >>>>
>> >>>
>> >>
>> >>
>> >
>>
>>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
> However, I don't understand your argument about putting aggregate()
> after the withXX() -- all the calls to withXX() set optional parameters
> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> be quite confusion for developers.
>
>
I see what you are saying, but the grouped stream is effectively a no-op
until you call one of the aggregate/count/reduce etc functions. So the
optional params are ones that are applicable to any of the operations you
can perform on this grouped stream. Then the final
count()/reduce()/aggregate() call has any of the params that are
required/specific to that function.


>
> -Matthias
>
> On 6/28/17 2:55 AM, Damian Guy wrote:
> >> I also think that mixing optional parameters with configs is a bad idea.
> >> Have not proposal for this atm but just wanted to mention it. Hope to
> >> find some time to come up with something.
> >>
> >>
> > Yes, i don't like the mix of config either. But the only real config here
> > is the logging config - which we don't really need as it can already be
> > done via a custom StateStoreSupplier.
> >
> >
> >> What I don't like in the current proposal is the
> >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> >> and .groupByKey() seems better. For clarity, we could rename to
> >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> >> some better names).
> >>
> >>
> > it could be groupByKey(), groupBy() or something different bt
> >
> >
> >
> >> The proposed pattern "chains" grouping and aggregation too close
> >> together. I would rather separate both more than less, ie, do into the
> >> opposite direction.
> >>
> >> I am also wondering, if we could so something more "fluent". The initial
> >> proposal was like:
> >>
> >>>> groupedStream.count()
> >>>>    .withStoreName("name")
> >>>>    .withCachingEnabled(false)
> >>>>    .withLoggingEnabled(config)
> >>>>    .table()
> >>
> >> The .table() statement in the end was kinda alien.
> >>
> >
> > I agree, but then all of the withXXX methods need to be on KTable which
> is
> > worse in my opinion. You also need something that is going to "build" the
> > internal processors and add them to the topology.
> >
> >
> >> The current proposal put the count() into the end -- ie, the optional
> >> parameter for count() have to specified on the .grouped() call -- this
> >> does not seems to be the best way either.
> >>
> >>
> > I actually prefer this method as you are building a grouped stream that
> you
> > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> etc
> > seems natural to me.
> >
> >
> >> I did not think this through in detail, but can't we just do the initial
> >> proposal with the .table() ?
> >>
> >> groupedStream.count().withStoreName("name").mapValues(...)
> >>
> >> Each .withXXX(...) return the current KTable and all the .withXXX() are
> >> just added to the KTable interface. Or do I miss anything why this wont'
> >> work or any obvious disadvantage?
> >>
> >>
> >>
> > See above.
> >
> >
> >>
> >> -Matthias
> >>
> >> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> >>> approach, but i think it is slightly nicer.
> >>> I agree with some of what Eno said about mixing configy stuff in the
> DSL,
> >>> but i think that enabling caching and enabling logging are things that
> >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> >>> though it is below) as this is actually config and we already have a
> way
> >> of
> >>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>> StateStoreSupplier for disabling caching etc, but as it stands that is
> a
> >>> bit of a tedious process for someone that just wants to use the default
> >>> storage engine, but not have caching enabled.
> >>>
> >>> There is also an orthogonal concern that Guozhang alluded to.... If you
> >>> want to plug in a custom storage engine and you want it to be logged
> etc,
> >>> you would currently need to implement that yourself. Ideally we can
> >> provide
> >>> a way where we will wrap the custom store with logging, metrics, etc. I
> >>> need to think about where this fits, it is probably more appropriate on
> >> the
> >>> Stores API.
> >>>
> >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>> // count with mapped key
> >>> final KTable<Long, Long> count = stream.grouped()
> >>>         .withKeyMapper(keyMapper)
> >>>         .withKeySerde(Serdes.Long())
> >>>         .withValueSerde(Serdes.String())
> >>>         .withQueryableName("my-store")
> >>>         .count();
> >>>
> >>> // windowed count
> >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .count();
> >>>
> >>> // windowed reduce
> >>> final Reducer<String> windowedReducer = null;
> >>> final KTable<Windowed<String>, String> windowedReduce =
> stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .reduce(windowedReducer);
> >>>
> >>> final Aggregator<String, String, Long> aggregator = null;
> >>> final Initializer<Long> init = null;
> >>>
> >>> // aggregate
> >>> final KTable<String, Long> aggregate = stream.grouped()
> >>>         .withQueryableName("my-aggregate-store")
> >>>         .aggregate(aggregator, init, Serdes.Long());
> >>>
> >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> stateStoreSupplier
> >> = null;
> >>>
> >>> // aggregate with custom store
> >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>         .aggregate(aggregator, init);
> >>>
> >>> // disable caching
> >>> stream.grouped()
> >>>         .withQueryableName("name")
> >>>         .withCachingEnabled(false)
> >>>         .count();
> >>>
> >>> // disable logging
> >>> stream.grouped()
> >>>         .withQueryableName("q")
> >>>         .withLoggingEnabled(false)
> >>>         .count();
> >>>
> >>> // override log config
> >>> final Reducer<String> reducer = null;
> >>> stream.grouped()
> >>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
> >>>         .reduce(reducer);
> >>>
> >>>
> >>> If anyone wants to play around with this you can find the code here:
> >>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>
> >>> Note: It won't actually work as most of the methods just return null.
> >>>
> >>> Thanks,
> >>> Damian
> >>>
> >>>
> >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> >>>
> >>>> Thanks Damian. I think both options have pros and cons. And both are
> >> better
> >>>> than overload abuse.
> >>>>
> >>>> The fluent API approach reads better, no mention of builder or build
> >>>> anywhere. The main downside is that the method signatures are a little
> >> less
> >>>> clear. By reading the method signature, one doesn't necessarily knows
> >> what
> >>>> it returns. Also, one needs to figure out the special method
> (`table()`
> >> in
> >>>> this case) that gives you what you actually care about (`KTable` in
> this
> >>>> case). Not major issues, but worth mentioning while doing the
> >> comparison.
> >>>>
> >>>> The builder approach avoids the issues mentioned above, but it doesn't
> >> read
> >>>> as well.
> >>>>
> >>>> Ismael
> >>>>
> >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> I'd like to get a discussion going around some of the API choices
> we've
> >>>>> made in the DLS. In particular those that relate to stateful
> operations
> >>>>> (though this could expand).
> >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> >> there
> >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> and
> >> i
> >>>>> feel it is only going to get worse as we add more optional params. In
> >>>>> particular we've had some requests to be able to turn caching off, or
> >>>>> change log configs,  on a per operator basis (note this can be done
> now
> >>>> if
> >>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>>>>
> >>>>> So this is a bit of an open question. How can we change the DSL
> >> overloads
> >>>>> so that it flows, is simple to use and understand, and is easily
> >> extended
> >>>>> in the future?
> >>>>>
> >>>>> One option would be to use a fluent API approach for providing the
> >>>> optional
> >>>>> params, so something like this:
> >>>>>
> >>>>> groupedStream.count()
> >>>>>    .withStoreName("name")
> >>>>>    .withCachingEnabled(false)
> >>>>>    .withLoggingEnabled(config)
> >>>>>    .table()
> >>>>>
> >>>>>
> >>>>>
> >>>>> Another option would be to provide a Builder to the count method, so
> it
> >>>>> would look something like this:
> >>>>> groupedStream.count(new
> >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>
> >>>>> Another option is to say: Hey we don't need this, what are you on
> >> about!
> >>>>>
> >>>>> The above has focussed on state store related overloads, but the same
> >>>> ideas
> >>>>> could  be applied to joins etc, where we presently have many join
> >> methods
> >>>>> and many overloads.
> >>>>>
> >>>>> Anyway, i look forward to hearing your opinions.
> >>>>>
> >>>>> Thanks,
> >>>>> Damian
> >>>>>
> >>>>
> >>>
> >>
> >>
> >
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Guozhang,

Thanks for the feedback

On Thu, 29 Jun 2017 at 01:45 Guozhang Wang <wa...@gmail.com> wrote:

> I played the current proposal a bit with https://github.com/dguy/kafka/
> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment>,
> and here are my observations:
>
> 1. Personally I prefer
>
>     "stream.group(mapper) / stream.groupByKey()"
>
> than
>
>     "stream.group().withKeyMapper(mapper) / stream.group()"
>
> Since 1) withKeyMapper is not enforced programmatically though it is not
> "really" optional like others, 2) syntax-wise it reads more natural.
>
>
Yes, that is true. The keyMapper is not optional to `group` so should be an
argument to the method


> I think it is okay to add the APIs in (
>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> )
> in KGroupedStream.
>
>
Are you saying add these methods to KGroupedStream? I'm not so sure as
they'd all need to return a KGroupedStream and then we would need to be
able to remove/replace/update processors in the topology (which would be
achievable, but i think not before KIP-120 is done so that we don't expose
any further public APIs that are only to support internal usages - also
updating processors in the topology seems a bit odd IMO). Perhaps you mean
something different.


>
> 2. For the "withStateStoreSupplier" API, are the user supposed to pass in
> the most-inner state store supplier (e.g. then one whose get() return
> RocksDBStore), or it is supposed to return the most-outer supplier with
> logging / metrics / etc? I think it would be more useful to only require
> users pass in the inner state store supplier while specifying caching /
> logging through other APIs.
>

Yes that is possible, though in that case we might want to use a narrower
interface than StateStoreSupplier, i.e., one that doesn't have `logConfig`
and `loggingEnabled`? Those two methods are exposed for the
`TopologyBuilder`


>
> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we are
> allowing users to call other APIs like "withQueryableName" multiple time,
> but only call "withStateStoreSupplier" only once in the end. Why is that?
>
>
The API is not meant to be complete! It is just an experiment to see what
is possible. However, at some point we need to decide which methods are
valid at any point. So if you have called `withStateStoreSupplier` you no
longer need to call `withQueryableName`, `withKeySerde`, `withValueSerde`
etc as this is already done.  The point being to narrow the interface to
what makes sense. There is work to be done here!


>
> 3. The current DSL seems to be only for aggregations, what about joins?
>
>
You have to start somewhere! I'd like to think we can come to an agreement
on a style and then apply that across the board. So the API should look the
same everywhere. But yes i can add some join examples.



>
> 4. I think it is okay to keep the "withLogConfig": for the
> StateStoreSupplier it will still be user code specifying the topology so I
> do not see there is a big difference.
>
>
I guess this one of the most contentious points, i.e., mixing config with
topology building. I'm on the fence on this as I get both sides of the
argument. I also don't really like having everything in config.


>
> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> windowed state store supplier to enforce typing?
>
>
Yes it would. I was just doing a bare bones API so at least you could try
it out.


>
> Below are minor ones:
>
> 6. "withQueryableName": maybe better "withQueryableStateName"?
>

I think we currently call it `queryableStoreName`. So maybe we stick with
that?


>
> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>
>
Yep



>
> Guozhang
>
>
>
> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > I see your point about "when to add the processor to the topology". That
> > is indeed an issue. Not sure it we could allow "updates" to the
> topology...
> >
> > I don't see any problem with having all the withXX() in KTable interface
> > -- but this might be subjective.
> >
> >
> > However, I don't understand your argument about putting aggregate()
> > after the withXX() -- all the calls to withXX() set optional parameters
> > for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > indicates that the withXX() belongs to the groupBy(). IMHO, this might
> > be quite confusion for developers.
> >
> >
> > -Matthias
> >
> > On 6/28/17 2:55 AM, Damian Guy wrote:
> > >> I also think that mixing optional parameters with configs is a bad
> idea.
> > >> Have not proposal for this atm but just wanted to mention it. Hope to
> > >> find some time to come up with something.
> > >>
> > >>
> > > Yes, i don't like the mix of config either. But the only real config
> here
> > > is the logging config - which we don't really need as it can already be
> > > done via a custom StateStoreSupplier.
> > >
> > >
> > >> What I don't like in the current proposal is the
> > >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> > >> and .groupByKey() seems better. For clarity, we could rename to
> > >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> > >> some better names).
> > >>
> > >>
> > > it could be groupByKey(), groupBy() or something different bt
> > >
> > >
> > >
> > >> The proposed pattern "chains" grouping and aggregation too close
> > >> together. I would rather separate both more than less, ie, do into the
> > >> opposite direction.
> > >>
> > >> I am also wondering, if we could so something more "fluent". The
> initial
> > >> proposal was like:
> > >>
> > >>>> groupedStream.count()
> > >>>>    .withStoreName("name")
> > >>>>    .withCachingEnabled(false)
> > >>>>    .withLoggingEnabled(config)
> > >>>>    .table()
> > >>
> > >> The .table() statement in the end was kinda alien.
> > >>
> > >
> > > I agree, but then all of the withXXX methods need to be on KTable which
> > is
> > > worse in my opinion. You also need something that is going to "build"
> the
> > > internal processors and add them to the topology.
> > >
> > >
> > >> The current proposal put the count() into the end -- ie, the optional
> > >> parameter for count() have to specified on the .grouped() call -- this
> > >> does not seems to be the best way either.
> > >>
> > >>
> > > I actually prefer this method as you are building a grouped stream that
> > you
> > > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> > etc
> > > seems natural to me.
> > >
> > >
> > >> I did not think this through in detail, but can't we just do the
> initial
> > >> proposal with the .table() ?
> > >>
> > >> groupedStream.count().withStoreName("name").mapValues(...)
> > >>
> > >> Each .withXXX(...) return the current KTable and all the .withXXX()
> are
> > >> just added to the KTable interface. Or do I miss anything why this
> wont'
> > >> work or any obvious disadvantage?
> > >>
> > >>
> > >>
> > > See above.
> > >
> > >
> > >>
> > >> -Matthias
> > >>
> > >> On 6/22/17 4:06 AM, Damian Guy wrote:
> > >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> > >>> approach, but i think it is slightly nicer.
> > >>> I agree with some of what Eno said about mixing configy stuff in the
> > DSL,
> > >>> but i think that enabling caching and enabling logging are things
> that
> > >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> > >>> though it is below) as this is actually config and we already have a
> > way
> > >> of
> > >>> doing that, via the StateStoreSupplier. Arguably we could use the
> > >>> StateStoreSupplier for disabling caching etc, but as it stands that
> is
> > a
> > >>> bit of a tedious process for someone that just wants to use the
> default
> > >>> storage engine, but not have caching enabled.
> > >>>
> > >>> There is also an orthogonal concern that Guozhang alluded to.... If
> you
> > >>> want to plug in a custom storage engine and you want it to be logged
> > etc,
> > >>> you would currently need to implement that yourself. Ideally we can
> > >> provide
> > >>> a way where we will wrap the custom store with logging, metrics,
> etc. I
> > >>> need to think about where this fits, it is probably more appropriate
> on
> > >> the
> > >>> Stores API.
> > >>>
> > >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > >>> // count with mapped key
> > >>> final KTable<Long, Long> count = stream.grouped()
> > >>>         .withKeyMapper(keyMapper)
> > >>>         .withKeySerde(Serdes.Long())
> > >>>         .withValueSerde(Serdes.String())
> > >>>         .withQueryableName("my-store")
> > >>>         .count();
> > >>>
> > >>> // windowed count
> > >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .count();
> > >>>
> > >>> // windowed reduce
> > >>> final Reducer<String> windowedReducer = null;
> > >>> final KTable<Windowed<String>, String> windowedReduce =
> > stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .reduce(windowedReducer);
> > >>>
> > >>> final Aggregator<String, String, Long> aggregator = null;
> > >>> final Initializer<Long> init = null;
> > >>>
> > >>> // aggregate
> > >>> final KTable<String, Long> aggregate = stream.grouped()
> > >>>         .withQueryableName("my-aggregate-store")
> > >>>         .aggregate(aggregator, init, Serdes.Long());
> > >>>
> > >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > stateStoreSupplier
> > >> = null;
> > >>>
> > >>> // aggregate with custom store
> > >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> > >>>         .withStateStoreSupplier(stateStoreSupplier)
> > >>>         .aggregate(aggregator, init);
> > >>>
> > >>> // disable caching
> > >>> stream.grouped()
> > >>>         .withQueryableName("name")
> > >>>         .withCachingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // disable logging
> > >>> stream.grouped()
> > >>>         .withQueryableName("q")
> > >>>         .withLoggingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // override log config
> > >>> final Reducer<String> reducer = null;
> > >>> stream.grouped()
> > >>>         .withLogConfig(Collections.singletonMap("segment.size",
> "10"))
> > >>>         .reduce(reducer);
> > >>>
> > >>>
> > >>> If anyone wants to play around with this you can find the code here:
> > >>> https://github.com/dguy/kafka/tree/dsl-experiment
> > >>>
> > >>> Note: It won't actually work as most of the methods just return null.
> > >>>
> > >>> Thanks,
> > >>> Damian
> > >>>
> > >>>
> > >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> > >>>
> > >>>> Thanks Damian. I think both options have pros and cons. And both are
> > >> better
> > >>>> than overload abuse.
> > >>>>
> > >>>> The fluent API approach reads better, no mention of builder or build
> > >>>> anywhere. The main downside is that the method signatures are a
> little
> > >> less
> > >>>> clear. By reading the method signature, one doesn't necessarily
> knows
> > >> what
> > >>>> it returns. Also, one needs to figure out the special method
> > (`table()`
> > >> in
> > >>>> this case) that gives you what you actually care about (`KTable` in
> > this
> > >>>> case). Not major issues, but worth mentioning while doing the
> > >> comparison.
> > >>>>
> > >>>> The builder approach avoids the issues mentioned above, but it
> doesn't
> > >> read
> > >>>> as well.
> > >>>>
> > >>>> Ismael
> > >>>>
> > >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> > >> wrote:
> > >>>>
> > >>>>> Hi,
> > >>>>>
> > >>>>> I'd like to get a discussion going around some of the API choices
> > we've
> > >>>>> made in the DLS. In particular those that relate to stateful
> > operations
> > >>>>> (though this could expand).
> > >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> > >> there
> > >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> > and
> > >> i
> > >>>>> feel it is only going to get worse as we add more optional params.
> In
> > >>>>> particular we've had some requests to be able to turn caching off,
> or
> > >>>>> change log configs,  on a per operator basis (note this can be done
> > now
> > >>>> if
> > >>>>> you pass in a StateStoreSupplier, but this can be a bit
> cumbersome).
> > >>>>>
> > >>>>> So this is a bit of an open question. How can we change the DSL
> > >> overloads
> > >>>>> so that it flows, is simple to use and understand, and is easily
> > >> extended
> > >>>>> in the future?
> > >>>>>
> > >>>>> One option would be to use a fluent API approach for providing the
> > >>>> optional
> > >>>>> params, so something like this:
> > >>>>>
> > >>>>> groupedStream.count()
> > >>>>>    .withStoreName("name")
> > >>>>>    .withCachingEnabled(false)
> > >>>>>    .withLoggingEnabled(config)
> > >>>>>    .table()
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Another option would be to provide a Builder to the count method,
> so
> > it
> > >>>>> would look something like this:
> > >>>>> groupedStream.count(new
> > >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>>>>
> > >>>>> Another option is to say: Hey we don't need this, what are you on
> > >> about!
> > >>>>>
> > >>>>> The above has focussed on state store related overloads, but the
> same
> > >>>> ideas
> > >>>>> could  be applied to joins etc, where we presently have many join
> > >> methods
> > >>>>> and many overloads.
> > >>>>>
> > >>>>> Anyway, i look forward to hearing your opinions.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Damian
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >>
> > >
> >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Mathias,

thanks again for your reply. Ill try to take some time now and make some 
more exhaustive remarks.

On 09.07.2017 21:48, Matthias J. Sax wrote:
> I think we do have a very good discussion and people openly share their
> ideas. So I am not sure why your are frustrated (at least I get this
> impression).
We do have good discussions. Its just that I am currently unhappy with 
the balance I make from:
"How bad I want my Ideas to be understood" vs "How much time I can throw 
in". That number is currently just through the roof! sorry that this 
makes it into the tone, really just my circumstances right now.

Additional to that: We have many features 1:n join, a different group by 
implementation, lateral view. All these things developed in 0.10.0.1 and 
have most of our stream processing needs covered now. Mostly with PAPI 
<=> DSL Switcharoo. We finally start an attempt to help upstreaming 
these feautures and coming up with some sort of blog post. And 0.10.0.1 
was like this little grass roots thing and sooo damn pretty. And I look 
at current trunk and there is quite some weeds growing and it really 
hurts me. More on this later I guess.
>
> Maybe it might be best if you propose an API change by yourself similar
> to what Damian and Guozhang did (to whatever extend your time constraint
> permits). I personally don't know exactly what an ideal API from your
> point of view is atm, but this discussion would benefit a lot if you
> could share it.
not entirely sure what to line out. Ill try at the end to give it a
>> I don't understand why custom stores in DSL?
> Why not? Maybe you can elaborate a little more?
A KTable is nothing by default: A KTable can _describe_ a mapping and a 
filtering from the join of 2 aggregates. Sources and Aggregates are the 
2 things that are something. They are special KTables not like a join.
When I check the newer codebase a join will now maintain a state store 
that is a copy of the 2 inputs (if its made queryiable with some 
nullchecks on strings). 2 reasons I can think we are there:
1. Less lookups on reads (wich kinda makes a point for High throughput 
IQ wich in my opionion becomes a task for Connect and materialize in 
some high throughput replicated DB)
2. It was to complicated to keep track of who has the serde for what and 
we didn't want to implement nested serialisation for all joined together 
things (wich is what a Ktable really is and I would have voted for that)
On the other to make it usable for a wide audience (we hope to get to 
hadoop grade adoption?) It probably needs to be configurable if you do 1 
or 2 as people can balance tradeoffs between more diskspace used
/ query latency / unnecessary transport (filter finds out its null after 
join)

When users put a Storename or a Supplier. Usually what they want to say 
is "I want to use it in IQ" (would love to hear counter arguments on 
that, honestly can't think of any). Back then when IQ was beeing 
investigated
I lengthly described my Idea of just putting KTable:getQueryHandle(). It 
would return a query handle wich implements get(K):V. A very basic 
approach would be to get the tables name and just wrap that around. 
People could store the
QueryHandle and after starting streams use it to access the tables data.
>
>> and I don't understand why
>>> we are not concidering a more generic config based appraoch?
> Not sure what you exactly mean. Sound interesting. I don't like the idea
> to mix configuration into the DSL (even if I am still not sure, where to
> draw the line, ie, what should we consider a config and what not).
Very difficult line to draw indeed. In a case to case evaluation i feel 
like finding reasonable answers, but I am very config heavy to be honest.
Having tons of Hive on my back, configs feel really neat. I can do all 
sorts of things by just setting options (run this in MR, this in TEZ, 
this on spark, no map side agg buffer please,  yada yada yada yada)
So I really do like the idea. Having the idea of "topology might change 
but semantics are preserved" sounds good for configs. Most of the 
Statestore stuff probably changes semantics, but usually in
crash situations (persistent(), enablelogging()). I am 100% with you 
that the decision for that is not easy and may be the most problematic 
point.
>
> About `through`: I think it does make sense to allow the specification
> of a store-name to make the store queryable (it's optional in 0.11 btw).
> It's the same as for `KStreamBuilder.table()` -- so not sure why this
> should be wrong?
getQueryHandle() would easily cover this case aswell. Whyever a table on 
the other side of a topic is worth more I don't quite get, some edgy 
repartitioning thing I guess. Just opening into many partitions
I would call to the connect guys again.
>
> Note, that not all KTables are materialized in a store atm. So it's an
> easy way to make a non-materialized KTable queryable.
I think that is inline with my comment above.
>
>>> also providing Serdes by config is neat. wouldn't even need to go into
>>> the code then would also save a ton. (We have the defaults one in conf
>>> why not override the specific ones?)
> I am not sure, if Serdes are really a config? I mean, the data types are
> hard coded into the code, so it does make sense to specify the Serdes
> accordingly. I am also not sure how we would map Serdes from the config
> to the corresponding operator?
true! maybe not an ideal case where configs help with overloading. I 
guess people are either using the global untyped one or a typed one for 
all steps.
So statestore is probably a better case. Its going to be referenced by a 
name always anyways so one could use this name to provide additional 
configs to the Statestore.
Probably also defining a factory used to build it.

Similarly a join has some sort of name, currently its 3 names, wich 
would need unifying to some degree, but then also the joins could be 
addressed with configs.
But Joins don't seem to have the to heaver overloading problem (Only 
store related :D).  But to be honest I can't judge the usefulness of 
outer and left. Not a pattern
that I came across yet for us its always inner. Maybe materialized but 
not sending old values is that what it does? Sorry can't wrap my head 
round that just now
heading towards 3am.

The example I provided was

streams.$applicationid.stores.$storename.inmemory = false
streams.$applicationid.stores.$storename.cachesize = 40k

for the configs. The Query Handle thing make sense hopefully.

Best Jan

>
> -Matthias
>
>
> On 7/8/17 2:23 AM, Jan Filipiak wrote:
>> Hi Matthias thanks,
>>
>> Exactly what I was guessing.
>>
>> I don't understand why custom stores in DSL? and I don't understand why
>> we are not concidering a more generic config based appraoch?
>>
>> StateStores in DSL => what I really think we are looking for PAPA => DSL
>> => PAPI  back and forth switcharoo capabilities.
>>
>> Looking at the most overloaded that I can currently find "through()" 2
>> of them come from the broken idea of "the user provides a name for the
>> statestore for IQ" and custom statestores.
>>  From the beginning I said that's madness. That is the real disease we
>> need to fix IMHO. To be honest I also don't understand why through with
>> statestore is particularly usefull, second Unique Key maybe?
>>
>> also providing Serdes by config is neat. wouldn't even need to go into
>> the code then would also save a ton. (We have the defaults one in conf
>> why not override the specific ones?)
>>
>> Does this makes sense to people? what pieces should i outline with code
>> (time is currently sparse :( but I can pull of some smaller examples i
>> guess)
>>
>> Best Jan
>>
>>
>>
>>
>>
>> On 08.07.2017 01:23, Matthias J. Sax wrote:
>>> It's too issues we want to tackle
>>>
>>>    - too many overload (for some method we have already more than 10(
>>>    - improve custom store API
>>>
>>> -Matthias
>>>
>>>
>>> On 7/7/17 3:42 PM, Jan Filipiak wrote:
>>>> It makes me want to cry.
>>>>
>>>> why on earth is the DSL going to expose all its implementation
>>>> details now?
>>>> especially being materialized or not.
>>>>
>>>> If we want to take usefull steps in that direction maybe we are looking
>>>> for a way to let the user switch back and forth between PAPI and DSL?
>>>>
>>>> A change as the proposed would not eliminate any of my pain points while
>>>> still being a heck of work migrating towards to.
>>>>
>>>> Since I am only following this from the point where Eno CC'ed it into
>>>> the users list:
>>>>
>>>> Can someone please rephrase for me what problem this is trying to solve?
>>>> I don't mean to be rude but It uses a problematic feature
>>>> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
>>>> us nowhere in making the configs more flexible, its just syntactic
>>>> sugar.
>>>>
>>>> A low effort shoot like: lets add a properties to operations that would
>>>> otherwise become overloaded to heavy? Or pull the configs by some naming
>>>> schema
>>>> form the overall properties. Additionally to that we get rid of
>>>> StateStoreSuppliers in the DSL and have them also configured by said
>>>> properties.
>>>>
>>>> => way easier to migrate to, way less risk, way more flexible in the
>>>> future (different implementations of the same operation don't require
>>>> code change to configure)
>>>>
>>>> Line 184 makes especially no sense to me. what is a KTableKTable non
>>>> materialized join anyways?
>>>>
>>>> Hope we can discuss more on this.
>>>>
>>>>
>>>>
>>>> On 07.07.2017 17:23, Guozhang Wang wrote:
>>>>> I messed the indentation on github code repos; this would be easier to
>>>>> read:
>>>>>
>>>>> https://codeshare.io/GLWW8K
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Damian / Kyle,
>>>>>>
>>>>>> I think I agree with you guys about the pros / cons of using the
>>>>>> builder
>>>>>> pattern v.s. using some "secondary classes". And I'm thinking if we
>>>>>> can
>>>>>> take a "mid" manner between these two. I spent some time with a slight
>>>>>> different approach from Damian's current proposal:
>>>>>>
>>>>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>>>>
>>>>>>
>>>>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>>>>
>>>>>> The key idea is to tolerate the final "table()" or "stream()"
>>>>>> function to
>>>>>> "upgrade" from the secondary classes to the first citizen classes,
>>>>>> while
>>>>>> having all the specs inside this function. Also this proposal
>>>>>> includes some
>>>>>> other refactoring that people have been discussed about for the
>>>>>> builder to
>>>>>> reduce the overloaded functions as well. WDYT?
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Jan,
>>>>>>>
>>>>>>> Thanks very much for the input.
>>>>>>>
>>>>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Damian,
>>>>>>>>
>>>>>>>> I do see your point of something needs to change. But I fully agree
>>>>>>>> with
>>>>>>>> Gouzhang when he says.
>>>>>>>> ---
>>>>>>>>
>>>>>>>> But since this is a incompatibility change, and we are going to
>>>>>>>> remove
>>>>>>> the
>>>>>>>> compatibility annotations soon it means we only have one chance
>>>>>>>> and we
>>>>>>>> really have to make it right.
>>>>>>>> ----
>>>>>>>>
>>>>>>>>
>>>>>>> I think we all agree on this one! Hence the discussion.
>>>>>>>
>>>>>>>
>>>>>>>> I fear all suggestions do not go far enough to become something that
>>>>>>> will
>>>>>>>> carry on for very much longer.
>>>>>>>> I am currently working on KAFKA-3705 and try to find the most
>>>>>>>> easy way
>>>>>>> for
>>>>>>>> the user to give me all the required functionality. The easiest
>>>>>>> interface I
>>>>>>>> could come up so far can be looked at here.
>>>>>>>>
>>>>>>>>
>>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>>>> And its already horribly complicated. I am currently unable to
>>>>>>> find the
>>>>>>>> right abstraction level to have everything falling into place
>>>>>>> naturally. To
>>>>>>>> be honest I already think introducing
>>>>>>>>
>>>>>>>>
>>>>>>> To be fair that is not a particularly easy problem to solve!
>>>>>>>
>>>>>>>
>>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>>>>> was unideal and makes everything a mess.
>>>>>>> I'm not sure i agree that it makes everything a mess, but It could
>>>>>>> have
>>>>>>> been done differently.
>>>>>>>
>>>>>>> The JoinType:Whatever is also not really flexible. 2 things come
>>>>>>> to my
>>>>>>> mind:
>>>>>>>> 1. I don't think we should rule out config based decisions say
>>>>>>>> configs
>>>>>>> like
>>>>>>>>            streams.$applicationID.joins.$joinname.conf = value
>>>>>>>>
>>>>>>> Is this just for config? Or are you suggesting that we could somehow
>>>>>>> "code"
>>>>>>> the join in a config file?
>>>>>>>
>>>>>>>
>>>>>>>> This can allow for tremendous changes without single API change and
>>>>>>>> IMO
>>>>>>> it
>>>>>>>> was not considered enough yet.
>>>>>>>>
>>>>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner
>>>>>>>> for
>>>>>>>> example can be used to implement different join types as the user
>>>>>>> wishes.
>>>>>>> Do you have an example of how this might look?
>>>>>>>
>>>>>>>
>>>>>>>> As Gouzhang said: stopping to break users is very important.
>>>>>>> Of course. We want to make it as easy as possible for people to use
>>>>>>> streams.
>>>>>>>
>>>>>>>
>>>>>>> especially with this changes + All the plans I sadly only have in my
>>>>>>> head
>>>>>>>> but hopefully the first link can give a glimpse.
>>>>>>>>
>>>>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>>>>> exactly
>>>>>>>> we are talking about. I would argue to go a bit slower and more
>>>>>>> carefull on
>>>>>>>> this one. At some point we need to get it right. Peeking over to the
>>>>>>> hadoop
>>>>>>>> guys with their hughe userbase. Config files really work well for
>>>>>>>> them.
>>>>>>>>
>>>>>>>> Best Jan
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>>>>> Thanks Matthias
>>>>>>>>>
>>>>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>>>>>>>>> <ma...@confluent.io>
>>>>>>>> wrote:
>>>>>>>>>> I am just catching up on this thread, so sorry for the long
>>>>>>>>>> email in
>>>>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>>>>> always a
>>>>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>>>>> maybe
>>>>>>>>>> it helps other to get new ideas :)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>>>>> aggregate()
>>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>> parameters
>>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>>> might
>>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> I see what you are saying, but the grouped stream is
>>>>>>>>>>> effectively a
>>>>>>>> no-op
>>>>>>>>>>> until you call one of the aggregate/count/reduce etc
>>>>>>>>>>> functions. So
>>>>>>> the
>>>>>>>>>>> optional params are ones that are applicable to any of the
>>>>>>> operations
>>>>>>>> you
>>>>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>>>>> required/specific to that function.
>>>>>>>>>>>
>>>>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>>>>> If we
>>>>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>>>>
>>>>>>>>>> .groupBy().count().withXX().build()
>>>>>>>>>>
>>>>>>>>>> (with a better name for build() though)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>>>>> apply
>>>>>>>>> too all the aggregate functions. The terminal call being the actual
>>>>>>> type
>>>>>>>> of
>>>>>>>>> aggregation you want to do. I personally find this more natural
>>>>>>>>> than
>>>>>>>>> groupBy().count().withBlah().build()
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>>>>> sessionWindowed(). We should have one top-level `Windows`
>>>>>>>>>> interface
>>>>>>> that
>>>>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>>>>> single
>>>>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>>>>> separation of `SessionWindows` in the first place, and this
>>>>>>>>>> seems to
>>>>>>> be
>>>>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>>>>> introduced session windows)
>>>>>>>>>>
>>>>>>>>> Yes - true we should look into that.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>>>>> thus we
>>>>>>>>>> might also want to use windowBy() (instead of windowed()). Not
>>>>>>>>>> sure
>>>>>>> how
>>>>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Makes sense
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>>>>> think,
>>>>>>>>>> defining an inner/left/outer join is not an optional argument
>>>>>>>>>> but a
>>>>>>>>>> first class concept and should have a proper representation in the
>>>>>>> API
>>>>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>>>>> what
>>>>>>>> we
>>>>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> About the two join API proposals, the second one has too much
>>>>>>>>>> boiler
>>>>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>>>>> one
>>>>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>>>>> operator call, should have one parameter per mandatory argument
>>>>>>>>>> but
>>>>>>> your
>>>>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>>>>> call.
>>>>>>>>>> This is far from intuitive IMHO.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> This is the builder pattern, you only need one param as the builder
>>>>>>> has
>>>>>>>>> captured all of the required and optional arguments.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> The first join proposal also seems to align better with the
>>>>>>>>>> pattern
>>>>>>>>>> suggested for aggregations and having the same pattern for all
>>>>>>> operators
>>>>>>>>>> is important (as you stated already).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>>>>> the
>>>>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Coming back to the config vs optional parameter. What about
>>>>>>>>>> having a
>>>>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>>>>> thing
>>>>>>>> that
>>>>>>>>> is really config.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> This also raises the question if until() is a windows property?
>>>>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>>>>> should
>>>>>>>>>> not not have it's own method.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>>>>> to be
>>>>>>>>> potentially different for every window operation you do in a
>>>>>>>>> streams
>>>>>>> app.
>>>>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>>>>
>>>>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>>>>      groupedStream.counting()
>>>>>>>>>>>                      .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>                      .table();
>>>>>>>>>> This is an interesting idea, and it remind my on some feedback
>>>>>>>>>> about
>>>>>>> "I
>>>>>>>>>> wanted to count a stream, but there was no count() method -- I
>>>>>>>>>> first
>>>>>>>>>> needed to figure out, that I need to group the stream first to be
>>>>>>> able
>>>>>>>>>> to count it. It does make sense in hindsight but was not
>>>>>>>>>> obvious in
>>>>>>> the
>>>>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>>>>> following:
>>>>>>>>>>
>>>>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>>>>
>>>>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative
>>>>>>>>>> here,
>>>>>>> as
>>>>>>>>>> it comes after the count()
>>>>>>>>>>
>>>>>>>>>> This would be more consistent than your proposal (that has
>>>>>>>>>> grouping
>>>>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>>>>> enrich
>>>>>>>>>> the API with a some syntactic sugar like
>>>>>>>>>> `stream.count().table()` to
>>>>>>> get
>>>>>>>>>> the overall count of all records (this would obviously not scale,
>>>>>>> but we
>>>>>>>>>> could support it -- if not now, maybe later).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> I guess i'd prefer
>>>>>>>>> stream.groupBy().windowBy().count()
>>>>>>>>> stream.groupBy().windowBy().reduce()
>>>>>>>>> stream.groupBy().count()
>>>>>>>>>
>>>>>>>>> As i said above, everything that happens before the final aggregate
>>>>>>> call
>>>>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>>>>> things
>>>>>>>>> ahead of the final aggregate call.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Last about builder pattern. I am convinced that we need some
>>>>>>> "terminal"
>>>>>>>>>> operator/method that tells us when to add the processor to the
>>>>>>> topology.
>>>>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>>>>> alien to
>>>>>>>>>> me (see my argument about the second join proposal). Using
>>>>>>>>>> .stream()
>>>>>>> /
>>>>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>>>>> generic
>>>>>>>>>> name that we can use in all places like build() or apply() might
>>>>>>> also be
>>>>>>>>>> an option.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Sure, a generic name might be ok.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>>>>> Thanks Kyle.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Hi Damian,
>>>>>>>>>>>>
>>>>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>>> discussed
>>>>>>>>>> most
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>>> you
>>>>>>>> can
>>>>>>>>>>>> reuse.
>>>>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse,
>>>>>>>>>>>>>>>> is it
>>>>>>>> legal
>>>>>>>>>> to
>>>>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped
>>>>>>>>>>>>>>>> each
>>>>>>>> time?
>>>>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>>> re-use it
>>>>>>>>>> as
>>>>>>>>>>>> you
>>>>>>>>>>>>>> can today.
>>>>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The
>>>>>>>>>>>> way I
>>>>>>> see
>>>>>>>>>> it
>>>>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>>>>> KTables,
>>>>>>>>>> because
>>>>>>>>>>>> most of this fluent api would continue returning this (this
>>>>>>>>>>>> being
>>>>>>> the
>>>>>>>>>>>> builder object currently being manipulated).
>>>>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>>>>> KTables
>>>>>>>>>> and
>>>>>>>>>>>> you use it in two different ways then its possible you make
>>>>>>>> conflicting
>>>>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> No necessarily true. It could return a new instance of the
>>>>>>>>>>> builder,
>>>>>>>> i.e.,
>>>>>>>>>>> the builders being immutable. So if you held a reference to the
>>>>>>> builder
>>>>>>>>>> it
>>>>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>>>>> kStream.grouped();
>>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going
>>>>>>>>>>>> to do
>>>>>>> it
>>>>>>>>>>>> eventually…
>>>>>>>>>>>> Depending on implementation uses of
>>>>>>>>>>>> groupedStreamWithDefaultSerdes
>>>>>>>> would
>>>>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>>>>> work
>>>>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>>>>> are
>>>>>>>>>>>> building, but this approach has its own problem because now we
>>>>>>> have to
>>>>>>>>>>>> identify which configs are equivalent so we don’t create
>>>>>>>>>>>> repeated
>>>>>>>>>>>> processors.
>>>>>>>>>>>>
>>>>>>>>>>>> The point of this long winded example is that we always have
>>>>>>>>>>>> to be
>>>>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>>>>> user
>>>>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>>>>
>>>>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>>>>> could
>>>>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>>>>> // Count
>>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>>
>>>>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>
>>>>>>>>
>>>>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse
>>>>>>>>>>>> the
>>>>>>>> Count
>>>>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>>>>> store
>>>>>>>>>> named
>>>>>>>>>>>> “my-store” already exists.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>>>>
>>>>>>>>>>> Cheers,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Kyle
>>>>>>>>>>>>
>>>>>>>>>>>> From: Damian Guy
>>>>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>>
>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>>>>
>>>>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>>>>> against
>>>>>>>>>>>>> it. The reason I like it is because it makes it clear to the
>>>>>>>>>>>>> user
>>>>>>>> that
>>>>>>>>>> a
>>>>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>>>>> intermediate
>>>>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>>>>
>>>>>>>>>>>> Yes, that makes sense.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>>>>>>>>> discussed
>>>>>>> most
>>>>>>>>>> it
>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>>>>>>>>> you
>>>>>>> can
>>>>>>>>>>>> reuse.
>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>>> legal
>>>>>>>> to
>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>>> time?
>>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>>>>>>>> re-use
>>>>>>> it
>>>>>>>> as
>>>>>>>>>> you
>>>>>>>>>>>> can today.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>>>>> pattern
>>>>>>>> I
>>>>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>>>>> big
>>>>>>>> fan
>>>>>>>>>> of
>>>>>>>>>>>>> the grouped.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>>>>> and
>>>>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>>>>> want to
>>>>>>>>>> be
>>>>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>>>>> all
>>>>>>>>>> depends
>>>>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>>>>> (probably
>>>>>>>> 3)
>>>>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>>>>> KTable
>>>>>>>>>> with
>>>>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>>>>> sessionwindowed
>>>>>>>>>>>> for
>>>>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>>>>> Count,
>>>>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>>>>> builder:
>>>>>>>>>>>>> // Count
>>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>>
>>>>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>
>>>>>>>>>>>>> // Windowed Count
>>>>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>>>>
>>>>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>>>>> // Session Count
>>>>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>>>>
>>>>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> // Reduce
>>>>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>>>>
>>>>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>>>>
>>>>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>>>>> KTable<String, String> cogrouped =
>>>>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>>>>             .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>>>>             .aggregate(initializer, aggregator,
>>>>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> public class Count {
>>>>>>>>>>>>>
>>>>>>>>>>>>>         public static class Windowed extends Count {
>>>>>>>>>>>>>             private Windows windows;
>>>>>>>>>>>>>         }
>>>>>>>>>>>>>         public static class SessionWindowed extends Count {
>>>>>>>>>>>>>             private SessionWindows sessionWindows;
>>>>>>>>>>>>>         }
>>>>>>>>>>>>>
>>>>>>>>>>>>>         public static Count count();
>>>>>>>>>>>>>         public static Windowed windowed(Windows windows);
>>>>>>>>>>>>>         public static SessionWindowed
>>>>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>>>>> sessionWindows);
>>>>>>>>>>>>>
>>>>>>>>>>>>>         // All withXXX(...) methods.
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>>>>         public KTable<K, Long> count(Count count);
>>>>>>>>>>>>>         public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>>>>> count);
>>>>>>>>>>>>>         public KTable<Windowed<K>, Long>
>>>>>>>>>>>>> count(Count.SessionWindowed
>>>>>>>>>> count);
>>>>>>>>>>>>> …
>>>>>>>>>>>>> }
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Kyle
>>>>>>>>>>>>>
>>>>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>>>
>>>>>>>>>>>>> I played the current proposal a bit with
>>>>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>>>>> tree/dsl-experiment <
>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>> ,
>>>>>>>>>>>>> and here are my observations:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>>>>
>>>>>>>>>>>>>         "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>>>>
>>>>>>>>>>>>> than
>>>>>>>>>>>>>
>>>>>>>>>>>>>         "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>>>>
>>>>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically
>>>>>>>>>>>>> though it
>>>>>>> is
>>>>>>>>>> not
>>>>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>>>>> natural.
>>>>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>>>>> )
>>>>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user
>>>>>>>>>>>>> supposed to
>>>>>>>> pass
>>>>>>>>>> in
>>>>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>>>>> return
>>>>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>>>>> supplier
>>>>>>>> with
>>>>>>>>>>>>> logging / metrics / etc? I think it would be more useful to
>>>>>>>>>>>>> only
>>>>>>>>>> require
>>>>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>>>>> caching /
>>>>>>>>>>>>> logging through other APIs.
>>>>>>>>>>>>>
>>>>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit
>>>>>>>>>>>>> suspicious to
>>>>>>> me:
>>>>>>>> we
>>>>>>>>>>>> are
>>>>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>>>>> multiple
>>>>>>>>>> time,
>>>>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end.
>>>>>>>>>>>>> Why
>>>>>>> is
>>>>>>>>>> that?
>>>>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what
>>>>>>>>>>>>> about
>>>>>>>> joins?
>>>>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>>>>> topology
>>>>>>>>>> so
>>>>>>>>>>>> I
>>>>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should
>>>>>>>>>>>>> take
>>>>>>> the
>>>>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Below are minor ones:
>>>>>>>>>>>>>
>>>>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>>>>
>>>>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I see your point about "when to add the processor to the
>>>>>>> topology".
>>>>>>>>>>>> That
>>>>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to
>>>>>>>>>>>>>> the
>>>>>>>>>>>>> topology...
>>>>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>>>>> interface
>>>>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>> aggregate()
>>>>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>>>> parameters
>>>>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO,
>>>>>>>>>>>>>> this
>>>>>>>> might
>>>>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>>>>> I also think that mixing optional parameters with configs
>>>>>>>>>>>>>>>> is a
>>>>>>> bad
>>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention
>>>>>>>>>>>>>>>> it.
>>>>>>> Hope
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>>>>> config
>>>>>>>>>>>>> here
>>>>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>>>>> already
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>>>>> rename to
>>>>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>>>>> should
>>>>>>>> find
>>>>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>>>>> close
>>>>>>>>>>>>>>>> together. I would rather separate both more than less,
>>>>>>>>>>>>>>>> ie, do
>>>>>>> into
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>>>>> The
>>>>>>>>>>>>> initial
>>>>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>>        .withStoreName("name")
>>>>>>>>>>>>>>>>>>        .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>>        .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>>        .table()
>>>>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>>>>> KTable
>>>>>>>>>>>> which
>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>>>>> "build"
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>> parameter for count() have to specified on the .grouped()
>>>>>>>>>>>>>>>> call
>>>>>>> -- 
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>>>>> stream
>>>>>>>>>>>> that
>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>> will aggregate. So
>>>>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>>>>> etc
>>>>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>>>>> the
>>>>>>>>>>>>> initial
>>>>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>>>>> .withXXX()
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything
>>>>>>>>>>>>>>>> why
>>>>>>> this
>>>>>>>>>>>>> wont'
>>>>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> See above.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds
>>>>>>>>>>>>>>>>> on the
>>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy
>>>>>>>>>>>>>>>>> stuff
>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>> DSL,
>>>>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>>>>> things
>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>>>>> withLogConfig(...)
>>>>>>>>>>>> (even
>>>>>>>>>>>>>>>>> though it is below) as this is actually config and we
>>>>>>>>>>>>>>>>> already
>>>>>>>> have
>>>>>>>>>>>> a
>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we
>>>>>>>>>>>>>>>>> could use
>>>>>>> the
>>>>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>>>>> stands
>>>>>>>> that
>>>>>>>>>>>>> is
>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>>>>> the
>>>>>>>>>>>>> default
>>>>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>>>>> to....
>>>>>>>> If
>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it
>>>>>>>>>>>>>>>>> to be
>>>>>>>>>>>> logged
>>>>>>>>>>>>>> etc,
>>>>>>>>>>>>>>>>> you would currently need to implement that yourself.
>>>>>>>>>>>>>>>>> Ideally
>>>>>>> we
>>>>>>>> can
>>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>>>>> metrics,
>>>>>>>>>>>>> etc. I
>>>>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>>>>> appropriate
>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper =
>>>>>>>>>>>>>>>>> null;
>>>>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>>>>             .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>>>>             .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>>>>             .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>>>>             .withQueryableName("my-store")
>>>>>>>>>>>>>>>>>             .count();
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>>             .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>>             .count();
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>>             .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>>             .reduce(windowedReducer);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>>>>             .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>>>>             .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>>>>             .aggregate(aggregator, init);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withQueryableName("name")
>>>>>>>>>>>>>>>>>             .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>             .count();
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withQueryableName("q")
>>>>>>>>>>>>>>>>>             .withLoggingEnabled(false)
>>>>>>>>>>>>>>>>>             .count();
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>>             .withLogConfig(Collections.sin
>>>>>>> gletonMap("segment.size",
>>>>>>>>>>>>> "10"))
>>>>>>>>>>>>>>>>>             .reduce(reducer);
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>>>>> code
>>>>>>>>>>>> here:
>>>>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>>>>> return
>>>>>>>>>>>> null.
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma
>>>>>>>>>>>>>>>>> <is...@juma.me.uk>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons.
>>>>>>>>>>>>>>>>>> And
>>>>>>> both
>>>>>>>>>>>> are
>>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of
>>>>>>>>>>>>>>>>>> builder
>>>>>>> or
>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>>>>> are a
>>>>>>>>>>>>> little
>>>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>>>>> necessarily
>>>>>>>>>>>>> knows
>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special
>>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>>> (`table()`
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>>>>> (`KTable`
>>>>>>>>>>>> in
>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above,
>>>>>>>>>>>>>>>>>> but
>>>>>>> it
>>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>>>>> choices
>>>>>>>>>>>>>> we've
>>>>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>>>>> API,
>>>>>>>>>>>> i.e,
>>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>>>>> becoming
>>>>>>>>>>>> noisy
>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more
>>>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>> params.
>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>>>>> caching
>>>>>>>>>>>> off,
>>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note
>>>>>>>>>>>>>>>>>>> this can
>>>>>>> be
>>>>>>>>>>>> done
>>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we
>>>>>>>>>>>>>>>>>>> change the
>>>>>>> DSL
>>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>>>>> easily
>>>>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>>>>> providing
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>>>        .withStoreName("name")
>>>>>>>>>>>>>>>>>>>        .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>>>        .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>>>        .table()
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>>>>> method,
>>>>>>>>>>>>> so
>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what
>>>>>>>>>>>>>>>>>>> are
>>>>>>> you
>>>>>>>> on
>>>>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>>>>> but
>>>>>>>> the
>>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>>>>> many
>>>>>>>> join
>>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>> -- 
>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>> -- 
>>>>>> -- Guozhang
>>>>>>
>>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I think we do have a very good discussion and people openly share their
ideas. So I am not sure why your are frustrated (at least I get this
impression).

Maybe it might be best if you propose an API change by yourself similar
to what Damian and Guozhang did (to whatever extend your time constraint
permits). I personally don't know exactly what an ideal API from your
point of view is atm, but this discussion would benefit a lot if you
could share it.

> I don't understand why custom stores in DSL?

Why not? Maybe you can elaborate a little more?

> and I don't understand why
>> we are not concidering a more generic config based appraoch?

Not sure what you exactly mean. Sound interesting. I don't like the idea
to mix configuration into the DSL (even if I am still not sure, where to
draw the line, ie, what should we consider a config and what not).

About `through`: I think it does make sense to allow the specification
of a store-name to make the store queryable (it's optional in 0.11 btw).
It's the same as for `KStreamBuilder.table()` -- so not sure why this
should be wrong?

Note, that not all KTables are materialized in a store atm. So it's an
easy way to make a non-materialized KTable queryable.

>> also providing Serdes by config is neat. wouldn't even need to go into
>> the code then would also save a ton. (We have the defaults one in conf
>> why not override the specific ones?)

I am not sure, if Serdes are really a config? I mean, the data types are
hard coded into the code, so it does make sense to specify the Serdes
accordingly. I am also not sure how we would map Serdes from the config
to the corresponding operator?


-Matthias


On 7/8/17 2:23 AM, Jan Filipiak wrote:
> Hi Matthias thanks,
> 
> Exactly what I was guessing.
> 
> I don't understand why custom stores in DSL? and I don't understand why
> we are not concidering a more generic config based appraoch?
> 
> StateStores in DSL => what I really think we are looking for PAPA => DSL
> => PAPI  back and forth switcharoo capabilities.
> 
> Looking at the most overloaded that I can currently find "through()" 2
> of them come from the broken idea of "the user provides a name for the
> statestore for IQ" and custom statestores.
> From the beginning I said that's madness. That is the real disease we
> need to fix IMHO. To be honest I also don't understand why through with
> statestore is particularly usefull, second Unique Key maybe?
> 
> also providing Serdes by config is neat. wouldn't even need to go into
> the code then would also save a ton. (We have the defaults one in conf
> why not override the specific ones?)
> 
> Does this makes sense to people? what pieces should i outline with code
> (time is currently sparse :( but I can pull of some smaller examples i
> guess)
> 
> Best Jan
> 
> 
> 
> 
> 
> On 08.07.2017 01:23, Matthias J. Sax wrote:
>> It's too issues we want to tackle
>>
>>   - too many overload (for some method we have already more than 10(
>>   - improve custom store API
>>
>> -Matthias
>>
>>
>> On 7/7/17 3:42 PM, Jan Filipiak wrote:
>>> It makes me want to cry.
>>>
>>> why on earth is the DSL going to expose all its implementation
>>> details now?
>>> especially being materialized or not.
>>>
>>> If we want to take usefull steps in that direction maybe we are looking
>>> for a way to let the user switch back and forth between PAPI and DSL?
>>>
>>> A change as the proposed would not eliminate any of my pain points while
>>> still being a heck of work migrating towards to.
>>>
>>> Since I am only following this from the point where Eno CC'ed it into
>>> the users list:
>>>
>>> Can someone please rephrase for me what problem this is trying to solve?
>>> I don't mean to be rude but It uses a problematic feature
>>> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
>>> us nowhere in making the configs more flexible, its just syntactic
>>> sugar.
>>>
>>> A low effort shoot like: lets add a properties to operations that would
>>> otherwise become overloaded to heavy? Or pull the configs by some naming
>>> schema
>>> form the overall properties. Additionally to that we get rid of
>>> StateStoreSuppliers in the DSL and have them also configured by said
>>> properties.
>>>
>>> => way easier to migrate to, way less risk, way more flexible in the
>>> future (different implementations of the same operation don't require
>>> code change to configure)
>>>
>>> Line 184 makes especially no sense to me. what is a KTableKTable non
>>> materialized join anyways?
>>>
>>> Hope we can discuss more on this.
>>>
>>>
>>>
>>> On 07.07.2017 17:23, Guozhang Wang wrote:
>>>> I messed the indentation on github code repos; this would be easier to
>>>> read:
>>>>
>>>> https://codeshare.io/GLWW8K
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi Damian / Kyle,
>>>>>
>>>>> I think I agree with you guys about the pros / cons of using the
>>>>> builder
>>>>> pattern v.s. using some "secondary classes". And I'm thinking if we
>>>>> can
>>>>> take a "mid" manner between these two. I spent some time with a slight
>>>>> different approach from Damian's current proposal:
>>>>>
>>>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>>>
>>>>>
>>>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>>>
>>>>> The key idea is to tolerate the final "table()" or "stream()"
>>>>> function to
>>>>> "upgrade" from the secondary classes to the first citizen classes,
>>>>> while
>>>>> having all the specs inside this function. Also this proposal
>>>>> includes some
>>>>> other refactoring that people have been discussed about for the
>>>>> builder to
>>>>> reduce the overloaded functions as well. WDYT?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Jan,
>>>>>>
>>>>>> Thanks very much for the input.
>>>>>>
>>>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Damian,
>>>>>>>
>>>>>>> I do see your point of something needs to change. But I fully agree
>>>>>>> with
>>>>>>> Gouzhang when he says.
>>>>>>> ---
>>>>>>>
>>>>>>> But since this is a incompatibility change, and we are going to
>>>>>>> remove
>>>>>> the
>>>>>>> compatibility annotations soon it means we only have one chance
>>>>>>> and we
>>>>>>> really have to make it right.
>>>>>>> ----
>>>>>>>
>>>>>>>
>>>>>> I think we all agree on this one! Hence the discussion.
>>>>>>
>>>>>>
>>>>>>> I fear all suggestions do not go far enough to become something that
>>>>>> will
>>>>>>> carry on for very much longer.
>>>>>>> I am currently working on KAFKA-3705 and try to find the most
>>>>>>> easy way
>>>>>> for
>>>>>>> the user to give me all the required functionality. The easiest
>>>>>> interface I
>>>>>>> could come up so far can be looked at here.
>>>>>>>
>>>>>>>
>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>>> And its already horribly complicated. I am currently unable to
>>>>>> find the
>>>>>>> right abstraction level to have everything falling into place
>>>>>> naturally. To
>>>>>>> be honest I already think introducing
>>>>>>>
>>>>>>>
>>>>>> To be fair that is not a particularly easy problem to solve!
>>>>>>
>>>>>>
>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>>>> was unideal and makes everything a mess.
>>>>>> I'm not sure i agree that it makes everything a mess, but It could
>>>>>> have
>>>>>> been done differently.
>>>>>>
>>>>>> The JoinType:Whatever is also not really flexible. 2 things come
>>>>>> to my
>>>>>> mind:
>>>>>>> 1. I don't think we should rule out config based decisions say
>>>>>>> configs
>>>>>> like
>>>>>>>           streams.$applicationID.joins.$joinname.conf = value
>>>>>>>
>>>>>> Is this just for config? Or are you suggesting that we could somehow
>>>>>> "code"
>>>>>> the join in a config file?
>>>>>>
>>>>>>
>>>>>>> This can allow for tremendous changes without single API change and
>>>>>>> IMO
>>>>>> it
>>>>>>> was not considered enough yet.
>>>>>>>
>>>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner
>>>>>>> for
>>>>>>> example can be used to implement different join types as the user
>>>>>> wishes.
>>>>>> Do you have an example of how this might look?
>>>>>>
>>>>>>
>>>>>>> As Gouzhang said: stopping to break users is very important.
>>>>>> Of course. We want to make it as easy as possible for people to use
>>>>>> streams.
>>>>>>
>>>>>>
>>>>>> especially with this changes + All the plans I sadly only have in my
>>>>>> head
>>>>>>> but hopefully the first link can give a glimpse.
>>>>>>>
>>>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>>>> exactly
>>>>>>> we are talking about. I would argue to go a bit slower and more
>>>>>> carefull on
>>>>>>> this one. At some point we need to get it right. Peeking over to the
>>>>>> hadoop
>>>>>>> guys with their hughe userbase. Config files really work well for
>>>>>>> them.
>>>>>>>
>>>>>>> Best Jan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>>>> Thanks Matthias
>>>>>>>>
>>>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>>>>>>>> <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>>> I am just catching up on this thread, so sorry for the long
>>>>>>>>> email in
>>>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>>>> always a
>>>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>>>> maybe
>>>>>>>>> it helps other to get new ideas :)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>>>> aggregate()
>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>> parameters
>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>> might
>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> I see what you are saying, but the grouped stream is
>>>>>>>>>> effectively a
>>>>>>> no-op
>>>>>>>>>> until you call one of the aggregate/count/reduce etc
>>>>>>>>>> functions. So
>>>>>> the
>>>>>>>>>> optional params are ones that are applicable to any of the
>>>>>> operations
>>>>>>> you
>>>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>>>> required/specific to that function.
>>>>>>>>>>
>>>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>>>> If we
>>>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>>>
>>>>>>>>> .groupBy().count().withXX().build()
>>>>>>>>>
>>>>>>>>> (with a better name for build() though)
>>>>>>>>>
>>>>>>>>>
>>>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>>>> apply
>>>>>>>> too all the aggregate functions. The terminal call being the actual
>>>>>> type
>>>>>>> of
>>>>>>>> aggregation you want to do. I personally find this more natural
>>>>>>>> than
>>>>>>>> groupBy().count().withBlah().build()
>>>>>>>>
>>>>>>>>
>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>>>> sessionWindowed(). We should have one top-level `Windows`
>>>>>>>>> interface
>>>>>> that
>>>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>>>> single
>>>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>>>> separation of `SessionWindows` in the first place, and this
>>>>>>>>> seems to
>>>>>> be
>>>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>>>> introduced session windows)
>>>>>>>>>
>>>>>>>> Yes - true we should look into that.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>>>> thus we
>>>>>>>>> might also want to use windowBy() (instead of windowed()). Not
>>>>>>>>> sure
>>>>>> how
>>>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Makes sense
>>>>>>>>
>>>>>>>>
>>>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>>>> think,
>>>>>>>>> defining an inner/left/outer join is not an optional argument
>>>>>>>>> but a
>>>>>>>>> first class concept and should have a proper representation in the
>>>>>> API
>>>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>>>> what
>>>>>>> we
>>>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> About the two join API proposals, the second one has too much
>>>>>>>>> boiler
>>>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>>>> one
>>>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>>>> operator call, should have one parameter per mandatory argument
>>>>>>>>> but
>>>>>> your
>>>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>>>> call.
>>>>>>>>> This is far from intuitive IMHO.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> This is the builder pattern, you only need one param as the builder
>>>>>> has
>>>>>>>> captured all of the required and optional arguments.
>>>>>>>>
>>>>>>>>
>>>>>>>>> The first join proposal also seems to align better with the
>>>>>>>>> pattern
>>>>>>>>> suggested for aggregations and having the same pattern for all
>>>>>> operators
>>>>>>>>> is important (as you stated already).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>>>> the
>>>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Coming back to the config vs optional parameter. What about
>>>>>>>>> having a
>>>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>>>> thing
>>>>>>> that
>>>>>>>> is really config.
>>>>>>>>
>>>>>>>>
>>>>>>>>> This also raises the question if until() is a windows property?
>>>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>>>> should
>>>>>>>>> not not have it's own method.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>>>> to be
>>>>>>>> potentially different for every window operation you do in a
>>>>>>>> streams
>>>>>> app.
>>>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>>>
>>>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>>>     groupedStream.counting()
>>>>>>>>>>                     .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>                     .table();
>>>>>>>>> This is an interesting idea, and it remind my on some feedback
>>>>>>>>> about
>>>>>> "I
>>>>>>>>> wanted to count a stream, but there was no count() method -- I
>>>>>>>>> first
>>>>>>>>> needed to figure out, that I need to group the stream first to be
>>>>>> able
>>>>>>>>> to count it. It does make sense in hindsight but was not
>>>>>>>>> obvious in
>>>>>> the
>>>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>>>> following:
>>>>>>>>>
>>>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>>>
>>>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative
>>>>>>>>> here,
>>>>>> as
>>>>>>>>> it comes after the count()
>>>>>>>>>
>>>>>>>>> This would be more consistent than your proposal (that has
>>>>>>>>> grouping
>>>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>>>> enrich
>>>>>>>>> the API with a some syntactic sugar like
>>>>>>>>> `stream.count().table()` to
>>>>>> get
>>>>>>>>> the overall count of all records (this would obviously not scale,
>>>>>> but we
>>>>>>>>> could support it -- if not now, maybe later).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> I guess i'd prefer
>>>>>>>> stream.groupBy().windowBy().count()
>>>>>>>> stream.groupBy().windowBy().reduce()
>>>>>>>> stream.groupBy().count()
>>>>>>>>
>>>>>>>> As i said above, everything that happens before the final aggregate
>>>>>> call
>>>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>>>> things
>>>>>>>> ahead of the final aggregate call.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Last about builder pattern. I am convinced that we need some
>>>>>> "terminal"
>>>>>>>>> operator/method that tells us when to add the processor to the
>>>>>> topology.
>>>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>>>> alien to
>>>>>>>>> me (see my argument about the second join proposal). Using
>>>>>>>>> .stream()
>>>>>> /
>>>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>>>> generic
>>>>>>>>> name that we can use in all places like build() or apply() might
>>>>>> also be
>>>>>>>>> an option.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Sure, a generic name might be ok.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>>>> Thanks Kyle.
>>>>>>>>>>
>>>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Damian,
>>>>>>>>>>>
>>>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>> discussed
>>>>>>>>> most
>>>>>>>>>>> it
>>>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>> you
>>>>>>> can
>>>>>>>>>>> reuse.
>>>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse,
>>>>>>>>>>>>>>> is it
>>>>>>> legal
>>>>>>>>> to
>>>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped
>>>>>>>>>>>>>>> each
>>>>>>> time?
>>>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>> re-use it
>>>>>>>>> as
>>>>>>>>>>> you
>>>>>>>>>>>>> can today.
>>>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The
>>>>>>>>>>> way I
>>>>>> see
>>>>>>>>> it
>>>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>>>> KTables,
>>>>>>>>> because
>>>>>>>>>>> most of this fluent api would continue returning this (this
>>>>>>>>>>> being
>>>>>> the
>>>>>>>>>>> builder object currently being manipulated).
>>>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>>>> KTables
>>>>>>>>> and
>>>>>>>>>>> you use it in two different ways then its possible you make
>>>>>>> conflicting
>>>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> No necessarily true. It could return a new instance of the
>>>>>>>>>> builder,
>>>>>>> i.e.,
>>>>>>>>>> the builders being immutable. So if you held a reference to the
>>>>>> builder
>>>>>>>>> it
>>>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>>>> kStream.grouped();
>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going
>>>>>>>>>>> to do
>>>>>> it
>>>>>>>>>>> eventually…
>>>>>>>>>>> Depending on implementation uses of
>>>>>>>>>>> groupedStreamWithDefaultSerdes
>>>>>>> would
>>>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>>>> work
>>>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>>>> are
>>>>>>>>>>> building, but this approach has its own problem because now we
>>>>>> have to
>>>>>>>>>>> identify which configs are equivalent so we don’t create
>>>>>>>>>>> repeated
>>>>>>>>>>> processors.
>>>>>>>>>>>
>>>>>>>>>>> The point of this long winded example is that we always have
>>>>>>>>>>> to be
>>>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>>>> user
>>>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>>>
>>>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>>>> could
>>>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>>>> // Count
>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>
>>>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>>
>>>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse
>>>>>>>>>>> the
>>>>>>> Count
>>>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>>>> store
>>>>>>>>> named
>>>>>>>>>>> “my-store” already exists.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Damian
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Kyle
>>>>>>>>>>>
>>>>>>>>>>> From: Damian Guy
>>>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>
>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>>>> against
>>>>>>>>>>>> it. The reason I like it is because it makes it clear to the
>>>>>>>>>>>> user
>>>>>>> that
>>>>>>>>> a
>>>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>>>> intermediate
>>>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>>>
>>>>>>>>>>> Yes, that makes sense.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>>>>>>>> discussed
>>>>>> most
>>>>>>>>> it
>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>>>>>>>> you
>>>>>> can
>>>>>>>>>>> reuse.
>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>> legal
>>>>>>> to
>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>> time?
>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>>>>>>> re-use
>>>>>> it
>>>>>>> as
>>>>>>>>> you
>>>>>>>>>>> can today.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>>>> pattern
>>>>>>> I
>>>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>>>> big
>>>>>>> fan
>>>>>>>>> of
>>>>>>>>>>>> the grouped.
>>>>>>>>>>>>
>>>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>>>> and
>>>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>>>> want to
>>>>>>>>> be
>>>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>>>> all
>>>>>>>>> depends
>>>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>>>> (probably
>>>>>>> 3)
>>>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>>>> KTable
>>>>>>>>> with
>>>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>>>> sessionwindowed
>>>>>>>>>>> for
>>>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>>>> Count,
>>>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>>>> builder:
>>>>>>>>>>>> // Count
>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>>>>>>> // Windowed Count
>>>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>>>> // Session Count
>>>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> // Reduce
>>>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>>>
>>>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>>>
>>>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>>>> KTable<String, String> cogrouped =
>>>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>>>            .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>>>            .aggregate(initializer, aggregator,
>>>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> public class Count {
>>>>>>>>>>>>
>>>>>>>>>>>>        public static class Windowed extends Count {
>>>>>>>>>>>>            private Windows windows;
>>>>>>>>>>>>        }
>>>>>>>>>>>>        public static class SessionWindowed extends Count {
>>>>>>>>>>>>            private SessionWindows sessionWindows;
>>>>>>>>>>>>        }
>>>>>>>>>>>>
>>>>>>>>>>>>        public static Count count();
>>>>>>>>>>>>        public static Windowed windowed(Windows windows);
>>>>>>>>>>>>        public static SessionWindowed
>>>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>>>> sessionWindows);
>>>>>>>>>>>>
>>>>>>>>>>>>        // All withXXX(...) methods.
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>>>        public KTable<K, Long> count(Count count);
>>>>>>>>>>>>        public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>>>> count);
>>>>>>>>>>>>        public KTable<Windowed<K>, Long>
>>>>>>>>>>>> count(Count.SessionWindowed
>>>>>>>>> count);
>>>>>>>>>>>> …
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Kyle
>>>>>>>>>>>>
>>>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>>
>>>>>>>>>>>> I played the current proposal a bit with
>>>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>>>> tree/dsl-experiment <
>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>> ,
>>>>>>>>>>>> and here are my observations:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>>>
>>>>>>>>>>>>        "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>>>
>>>>>>>>>>>> than
>>>>>>>>>>>>
>>>>>>>>>>>>        "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>>>
>>>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically
>>>>>>>>>>>> though it
>>>>>> is
>>>>>>>>> not
>>>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>>>> natural.
>>>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>>>> )
>>>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user
>>>>>>>>>>>> supposed to
>>>>>>> pass
>>>>>>>>> in
>>>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>>>> return
>>>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>>>> supplier
>>>>>>> with
>>>>>>>>>>>> logging / metrics / etc? I think it would be more useful to
>>>>>>>>>>>> only
>>>>>>>>> require
>>>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>>>> caching /
>>>>>>>>>>>> logging through other APIs.
>>>>>>>>>>>>
>>>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit
>>>>>>>>>>>> suspicious to
>>>>>> me:
>>>>>>> we
>>>>>>>>>>> are
>>>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>>>> multiple
>>>>>>>>> time,
>>>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end.
>>>>>>>>>>>> Why
>>>>>> is
>>>>>>>>> that?
>>>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what
>>>>>>>>>>>> about
>>>>>>> joins?
>>>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>>>> topology
>>>>>>>>> so
>>>>>>>>>>> I
>>>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should
>>>>>>>>>>>> take
>>>>>> the
>>>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Below are minor ones:
>>>>>>>>>>>>
>>>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>>>
>>>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I see your point about "when to add the processor to the
>>>>>> topology".
>>>>>>>>>>> That
>>>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to
>>>>>>>>>>>>> the
>>>>>>>>>>>> topology...
>>>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>>>> interface
>>>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>> aggregate()
>>>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>>> parameters
>>>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO,
>>>>>>>>>>>>> this
>>>>>>> might
>>>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>>>> I also think that mixing optional parameters with configs
>>>>>>>>>>>>>>> is a
>>>>>> bad
>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention
>>>>>>>>>>>>>>> it.
>>>>>> Hope
>>>>>>>>>>> to
>>>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>>>> config
>>>>>>>>>>>> here
>>>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>>>> already
>>>>>>>>>>> be
>>>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>>>> rename to
>>>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>>>> should
>>>>>>> find
>>>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>>>> close
>>>>>>>>>>>>>>> together. I would rather separate both more than less,
>>>>>>>>>>>>>>> ie, do
>>>>>> into
>>>>>>>>>>> the
>>>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>>>> The
>>>>>>>>>>>> initial
>>>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>>>> KTable
>>>>>>>>>>> which
>>>>>>>>>>>>> is
>>>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>>>> "build"
>>>>>>>>>>>> the
>>>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>>>> optional
>>>>>>>>>>>>>>> parameter for count() have to specified on the .grouped()
>>>>>>>>>>>>>>> call
>>>>>> -- 
>>>>>>>>>>> this
>>>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>>>> stream
>>>>>>>>>>> that
>>>>>>>>>>>>> you
>>>>>>>>>>>>>> will aggregate. So
>>>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>>>> etc
>>>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>>>> the
>>>>>>>>>>>> initial
>>>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>>>> .withXXX()
>>>>>>>>>>>> are
>>>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything
>>>>>>>>>>>>>>> why
>>>>>> this
>>>>>>>>>>>> wont'
>>>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> See above.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds
>>>>>>>>>>>>>>>> on the
>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy
>>>>>>>>>>>>>>>> stuff
>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>> DSL,
>>>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>>>> things
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>>>> withLogConfig(...)
>>>>>>>>>>> (even
>>>>>>>>>>>>>>>> though it is below) as this is actually config and we
>>>>>>>>>>>>>>>> already
>>>>>>> have
>>>>>>>>>>> a
>>>>>>>>>>>>> way
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we
>>>>>>>>>>>>>>>> could use
>>>>>> the
>>>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>>>> stands
>>>>>>> that
>>>>>>>>>>>> is
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>>>> the
>>>>>>>>>>>> default
>>>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>>>> to....
>>>>>>> If
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it
>>>>>>>>>>>>>>>> to be
>>>>>>>>>>> logged
>>>>>>>>>>>>> etc,
>>>>>>>>>>>>>>>> you would currently need to implement that yourself.
>>>>>>>>>>>>>>>> Ideally
>>>>>> we
>>>>>>> can
>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>>>> metrics,
>>>>>>>>>>>> etc. I
>>>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>>>> appropriate
>>>>>>>>>>>> on
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper =
>>>>>>>>>>>>>>>> null;
>>>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>>>            .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>>>            .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>>>            .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>>>            .withQueryableName("my-store")
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>            .reduce(windowedReducer);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>>>            .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>>>            .aggregate(aggregator, init);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("name")
>>>>>>>>>>>>>>>>            .withCachingEnabled(false)
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("q")
>>>>>>>>>>>>>>>>            .withLoggingEnabled(false)
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withLogConfig(Collections.sin
>>>>>> gletonMap("segment.size",
>>>>>>>>>>>> "10"))
>>>>>>>>>>>>>>>>            .reduce(reducer);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>>>> code
>>>>>>>>>>> here:
>>>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>>>> return
>>>>>>>>>>> null.
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma
>>>>>>>>>>>>>>>> <is...@juma.me.uk>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons.
>>>>>>>>>>>>>>>>> And
>>>>>> both
>>>>>>>>>>> are
>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of
>>>>>>>>>>>>>>>>> builder
>>>>>> or
>>>>>>>>>>> build
>>>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>>>> are a
>>>>>>>>>>>> little
>>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>>>> necessarily
>>>>>>>>>>>> knows
>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special
>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>> (`table()`
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>>>> (`KTable`
>>>>>>>>>>> in
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above,
>>>>>>>>>>>>>>>>> but
>>>>>> it
>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>>>> choices
>>>>>>>>>>>>> we've
>>>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>>>> API,
>>>>>>>>>>> i.e,
>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>>>> becoming
>>>>>>>>>>> noisy
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more
>>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>> params.
>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>>>> caching
>>>>>>>>>>> off,
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note
>>>>>>>>>>>>>>>>>> this can
>>>>>> be
>>>>>>>>>>> done
>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we
>>>>>>>>>>>>>>>>>> change the
>>>>>> DSL
>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>>>> easily
>>>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>>>> providing
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>>>> method,
>>>>>>>>>>>> so
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what
>>>>>>>>>>>>>>>>>> are
>>>>>> you
>>>>>>> on
>>>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>>>> but
>>>>>>> the
>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>>>> many
>>>>>>> join
>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> -- 
>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>
>>>>> -- 
>>>>> -- Guozhang
>>>>>
>>>>
>>>
> 
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I think we do have a very good discussion and people openly share their
ideas. So I am not sure why your are frustrated (at least I get this
impression).

Maybe it might be best if you propose an API change by yourself similar
to what Damian and Guozhang did (to whatever extend your time constraint
permits). I personally don't know exactly what an ideal API from your
point of view is atm, but this discussion would benefit a lot if you
could share it.

> I don't understand why custom stores in DSL?

Why not? Maybe you can elaborate a little more?

> and I don't understand why
>> we are not concidering a more generic config based appraoch?

Not sure what you exactly mean. Sound interesting. I don't like the idea
to mix configuration into the DSL (even if I am still not sure, where to
draw the line, ie, what should we consider a config and what not).

About `through`: I think it does make sense to allow the specification
of a store-name to make the store queryable (it's optional in 0.11 btw).
It's the same as for `KStreamBuilder.table()` -- so not sure why this
should be wrong?

Note, that not all KTables are materialized in a store atm. So it's an
easy way to make a non-materialized KTable queryable.

>> also providing Serdes by config is neat. wouldn't even need to go into
>> the code then would also save a ton. (We have the defaults one in conf
>> why not override the specific ones?)

I am not sure, if Serdes are really a config? I mean, the data types are
hard coded into the code, so it does make sense to specify the Serdes
accordingly. I am also not sure how we would map Serdes from the config
to the corresponding operator?


-Matthias


On 7/8/17 2:23 AM, Jan Filipiak wrote:
> Hi Matthias thanks,
> 
> Exactly what I was guessing.
> 
> I don't understand why custom stores in DSL? and I don't understand why
> we are not concidering a more generic config based appraoch?
> 
> StateStores in DSL => what I really think we are looking for PAPA => DSL
> => PAPI  back and forth switcharoo capabilities.
> 
> Looking at the most overloaded that I can currently find "through()" 2
> of them come from the broken idea of "the user provides a name for the
> statestore for IQ" and custom statestores.
> From the beginning I said that's madness. That is the real disease we
> need to fix IMHO. To be honest I also don't understand why through with
> statestore is particularly usefull, second Unique Key maybe?
> 
> also providing Serdes by config is neat. wouldn't even need to go into
> the code then would also save a ton. (We have the defaults one in conf
> why not override the specific ones?)
> 
> Does this makes sense to people? what pieces should i outline with code
> (time is currently sparse :( but I can pull of some smaller examples i
> guess)
> 
> Best Jan
> 
> 
> 
> 
> 
> On 08.07.2017 01:23, Matthias J. Sax wrote:
>> It's too issues we want to tackle
>>
>>   - too many overload (for some method we have already more than 10(
>>   - improve custom store API
>>
>> -Matthias
>>
>>
>> On 7/7/17 3:42 PM, Jan Filipiak wrote:
>>> It makes me want to cry.
>>>
>>> why on earth is the DSL going to expose all its implementation
>>> details now?
>>> especially being materialized or not.
>>>
>>> If we want to take usefull steps in that direction maybe we are looking
>>> for a way to let the user switch back and forth between PAPI and DSL?
>>>
>>> A change as the proposed would not eliminate any of my pain points while
>>> still being a heck of work migrating towards to.
>>>
>>> Since I am only following this from the point where Eno CC'ed it into
>>> the users list:
>>>
>>> Can someone please rephrase for me what problem this is trying to solve?
>>> I don't mean to be rude but It uses a problematic feature
>>> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
>>> us nowhere in making the configs more flexible, its just syntactic
>>> sugar.
>>>
>>> A low effort shoot like: lets add a properties to operations that would
>>> otherwise become overloaded to heavy? Or pull the configs by some naming
>>> schema
>>> form the overall properties. Additionally to that we get rid of
>>> StateStoreSuppliers in the DSL and have them also configured by said
>>> properties.
>>>
>>> => way easier to migrate to, way less risk, way more flexible in the
>>> future (different implementations of the same operation don't require
>>> code change to configure)
>>>
>>> Line 184 makes especially no sense to me. what is a KTableKTable non
>>> materialized join anyways?
>>>
>>> Hope we can discuss more on this.
>>>
>>>
>>>
>>> On 07.07.2017 17:23, Guozhang Wang wrote:
>>>> I messed the indentation on github code repos; this would be easier to
>>>> read:
>>>>
>>>> https://codeshare.io/GLWW8K
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi Damian / Kyle,
>>>>>
>>>>> I think I agree with you guys about the pros / cons of using the
>>>>> builder
>>>>> pattern v.s. using some "secondary classes". And I'm thinking if we
>>>>> can
>>>>> take a "mid" manner between these two. I spent some time with a slight
>>>>> different approach from Damian's current proposal:
>>>>>
>>>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>>>
>>>>>
>>>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>>>
>>>>> The key idea is to tolerate the final "table()" or "stream()"
>>>>> function to
>>>>> "upgrade" from the secondary classes to the first citizen classes,
>>>>> while
>>>>> having all the specs inside this function. Also this proposal
>>>>> includes some
>>>>> other refactoring that people have been discussed about for the
>>>>> builder to
>>>>> reduce the overloaded functions as well. WDYT?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Jan,
>>>>>>
>>>>>> Thanks very much for the input.
>>>>>>
>>>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi Damian,
>>>>>>>
>>>>>>> I do see your point of something needs to change. But I fully agree
>>>>>>> with
>>>>>>> Gouzhang when he says.
>>>>>>> ---
>>>>>>>
>>>>>>> But since this is a incompatibility change, and we are going to
>>>>>>> remove
>>>>>> the
>>>>>>> compatibility annotations soon it means we only have one chance
>>>>>>> and we
>>>>>>> really have to make it right.
>>>>>>> ----
>>>>>>>
>>>>>>>
>>>>>> I think we all agree on this one! Hence the discussion.
>>>>>>
>>>>>>
>>>>>>> I fear all suggestions do not go far enough to become something that
>>>>>> will
>>>>>>> carry on for very much longer.
>>>>>>> I am currently working on KAFKA-3705 and try to find the most
>>>>>>> easy way
>>>>>> for
>>>>>>> the user to give me all the required functionality. The easiest
>>>>>> interface I
>>>>>>> could come up so far can be looked at here.
>>>>>>>
>>>>>>>
>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>>> And its already horribly complicated. I am currently unable to
>>>>>> find the
>>>>>>> right abstraction level to have everything falling into place
>>>>>> naturally. To
>>>>>>> be honest I already think introducing
>>>>>>>
>>>>>>>
>>>>>> To be fair that is not a particularly easy problem to solve!
>>>>>>
>>>>>>
>>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>>>> was unideal and makes everything a mess.
>>>>>> I'm not sure i agree that it makes everything a mess, but It could
>>>>>> have
>>>>>> been done differently.
>>>>>>
>>>>>> The JoinType:Whatever is also not really flexible. 2 things come
>>>>>> to my
>>>>>> mind:
>>>>>>> 1. I don't think we should rule out config based decisions say
>>>>>>> configs
>>>>>> like
>>>>>>>           streams.$applicationID.joins.$joinname.conf = value
>>>>>>>
>>>>>> Is this just for config? Or are you suggesting that we could somehow
>>>>>> "code"
>>>>>> the join in a config file?
>>>>>>
>>>>>>
>>>>>>> This can allow for tremendous changes without single API change and
>>>>>>> IMO
>>>>>> it
>>>>>>> was not considered enough yet.
>>>>>>>
>>>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner
>>>>>>> for
>>>>>>> example can be used to implement different join types as the user
>>>>>> wishes.
>>>>>> Do you have an example of how this might look?
>>>>>>
>>>>>>
>>>>>>> As Gouzhang said: stopping to break users is very important.
>>>>>> Of course. We want to make it as easy as possible for people to use
>>>>>> streams.
>>>>>>
>>>>>>
>>>>>> especially with this changes + All the plans I sadly only have in my
>>>>>> head
>>>>>>> but hopefully the first link can give a glimpse.
>>>>>>>
>>>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>>>> exactly
>>>>>>> we are talking about. I would argue to go a bit slower and more
>>>>>> carefull on
>>>>>>> this one. At some point we need to get it right. Peeking over to the
>>>>>> hadoop
>>>>>>> guys with their hughe userbase. Config files really work well for
>>>>>>> them.
>>>>>>>
>>>>>>> Best Jan
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>>>> Thanks Matthias
>>>>>>>>
>>>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>>>>>>>> <ma...@confluent.io>
>>>>>>> wrote:
>>>>>>>>> I am just catching up on this thread, so sorry for the long
>>>>>>>>> email in
>>>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>>>> always a
>>>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>>>> maybe
>>>>>>>>> it helps other to get new ideas :)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>>>> aggregate()
>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>> parameters
>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>> might
>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> I see what you are saying, but the grouped stream is
>>>>>>>>>> effectively a
>>>>>>> no-op
>>>>>>>>>> until you call one of the aggregate/count/reduce etc
>>>>>>>>>> functions. So
>>>>>> the
>>>>>>>>>> optional params are ones that are applicable to any of the
>>>>>> operations
>>>>>>> you
>>>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>>>> required/specific to that function.
>>>>>>>>>>
>>>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>>>> If we
>>>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>>>
>>>>>>>>> .groupBy().count().withXX().build()
>>>>>>>>>
>>>>>>>>> (with a better name for build() though)
>>>>>>>>>
>>>>>>>>>
>>>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>>>> apply
>>>>>>>> too all the aggregate functions. The terminal call being the actual
>>>>>> type
>>>>>>> of
>>>>>>>> aggregation you want to do. I personally find this more natural
>>>>>>>> than
>>>>>>>> groupBy().count().withBlah().build()
>>>>>>>>
>>>>>>>>
>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>>>> sessionWindowed(). We should have one top-level `Windows`
>>>>>>>>> interface
>>>>>> that
>>>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>>>> single
>>>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>>>> separation of `SessionWindows` in the first place, and this
>>>>>>>>> seems to
>>>>>> be
>>>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>>>> introduced session windows)
>>>>>>>>>
>>>>>>>> Yes - true we should look into that.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>>>> thus we
>>>>>>>>> might also want to use windowBy() (instead of windowed()). Not
>>>>>>>>> sure
>>>>>> how
>>>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Makes sense
>>>>>>>>
>>>>>>>>
>>>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>>>> think,
>>>>>>>>> defining an inner/left/outer join is not an optional argument
>>>>>>>>> but a
>>>>>>>>> first class concept and should have a proper representation in the
>>>>>> API
>>>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>>>> what
>>>>>>> we
>>>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> About the two join API proposals, the second one has too much
>>>>>>>>> boiler
>>>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>>>> one
>>>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>>>> operator call, should have one parameter per mandatory argument
>>>>>>>>> but
>>>>>> your
>>>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>>>> call.
>>>>>>>>> This is far from intuitive IMHO.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> This is the builder pattern, you only need one param as the builder
>>>>>> has
>>>>>>>> captured all of the required and optional arguments.
>>>>>>>>
>>>>>>>>
>>>>>>>>> The first join proposal also seems to align better with the
>>>>>>>>> pattern
>>>>>>>>> suggested for aggregations and having the same pattern for all
>>>>>> operators
>>>>>>>>> is important (as you stated already).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>>>> the
>>>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Coming back to the config vs optional parameter. What about
>>>>>>>>> having a
>>>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>>>> thing
>>>>>>> that
>>>>>>>> is really config.
>>>>>>>>
>>>>>>>>
>>>>>>>>> This also raises the question if until() is a windows property?
>>>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>>>> should
>>>>>>>>> not not have it's own method.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>>>> to be
>>>>>>>> potentially different for every window operation you do in a
>>>>>>>> streams
>>>>>> app.
>>>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>>>
>>>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>>>     groupedStream.counting()
>>>>>>>>>>                     .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>                     .table();
>>>>>>>>> This is an interesting idea, and it remind my on some feedback
>>>>>>>>> about
>>>>>> "I
>>>>>>>>> wanted to count a stream, but there was no count() method -- I
>>>>>>>>> first
>>>>>>>>> needed to figure out, that I need to group the stream first to be
>>>>>> able
>>>>>>>>> to count it. It does make sense in hindsight but was not
>>>>>>>>> obvious in
>>>>>> the
>>>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>>>> following:
>>>>>>>>>
>>>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>>>
>>>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative
>>>>>>>>> here,
>>>>>> as
>>>>>>>>> it comes after the count()
>>>>>>>>>
>>>>>>>>> This would be more consistent than your proposal (that has
>>>>>>>>> grouping
>>>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>>>> enrich
>>>>>>>>> the API with a some syntactic sugar like
>>>>>>>>> `stream.count().table()` to
>>>>>> get
>>>>>>>>> the overall count of all records (this would obviously not scale,
>>>>>> but we
>>>>>>>>> could support it -- if not now, maybe later).
>>>>>>>>>
>>>>>>>>>
>>>>>>>> I guess i'd prefer
>>>>>>>> stream.groupBy().windowBy().count()
>>>>>>>> stream.groupBy().windowBy().reduce()
>>>>>>>> stream.groupBy().count()
>>>>>>>>
>>>>>>>> As i said above, everything that happens before the final aggregate
>>>>>> call
>>>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>>>> things
>>>>>>>> ahead of the final aggregate call.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Last about builder pattern. I am convinced that we need some
>>>>>> "terminal"
>>>>>>>>> operator/method that tells us when to add the processor to the
>>>>>> topology.
>>>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>>>> alien to
>>>>>>>>> me (see my argument about the second join proposal). Using
>>>>>>>>> .stream()
>>>>>> /
>>>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>>>> generic
>>>>>>>>> name that we can use in all places like build() or apply() might
>>>>>> also be
>>>>>>>>> an option.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Sure, a generic name might be ok.
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> -Matthias
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>>>> Thanks Kyle.
>>>>>>>>>>
>>>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi Damian,
>>>>>>>>>>>
>>>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>> discussed
>>>>>>>>> most
>>>>>>>>>>> it
>>>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>> you
>>>>>>> can
>>>>>>>>>>> reuse.
>>>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse,
>>>>>>>>>>>>>>> is it
>>>>>>> legal
>>>>>>>>> to
>>>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped
>>>>>>>>>>>>>>> each
>>>>>>> time?
>>>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>> re-use it
>>>>>>>>> as
>>>>>>>>>>> you
>>>>>>>>>>>>> can today.
>>>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The
>>>>>>>>>>> way I
>>>>>> see
>>>>>>>>> it
>>>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>>>> KTables,
>>>>>>>>> because
>>>>>>>>>>> most of this fluent api would continue returning this (this
>>>>>>>>>>> being
>>>>>> the
>>>>>>>>>>> builder object currently being manipulated).
>>>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>>>> KTables
>>>>>>>>> and
>>>>>>>>>>> you use it in two different ways then its possible you make
>>>>>>> conflicting
>>>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> No necessarily true. It could return a new instance of the
>>>>>>>>>> builder,
>>>>>>> i.e.,
>>>>>>>>>> the builders being immutable. So if you held a reference to the
>>>>>> builder
>>>>>>>>> it
>>>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>>>> kStream.grouped();
>>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going
>>>>>>>>>>> to do
>>>>>> it
>>>>>>>>>>> eventually…
>>>>>>>>>>> Depending on implementation uses of
>>>>>>>>>>> groupedStreamWithDefaultSerdes
>>>>>>> would
>>>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>>>> work
>>>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>>>> are
>>>>>>>>>>> building, but this approach has its own problem because now we
>>>>>> have to
>>>>>>>>>>> identify which configs are equivalent so we don’t create
>>>>>>>>>>> repeated
>>>>>>>>>>> processors.
>>>>>>>>>>>
>>>>>>>>>>> The point of this long winded example is that we always have
>>>>>>>>>>> to be
>>>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>>>> user
>>>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>>>
>>>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>>>> could
>>>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>>>> // Count
>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>
>>>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>>
>>>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse
>>>>>>>>>>> the
>>>>>>> Count
>>>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>>>> store
>>>>>>>>> named
>>>>>>>>>>> “my-store” already exists.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>>>
>>>>>>>>>> Cheers,
>>>>>>>>>> Damian
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Kyle
>>>>>>>>>>>
>>>>>>>>>>> From: Damian Guy
>>>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>
>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>
>>>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>>>
>>>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>>>> against
>>>>>>>>>>>> it. The reason I like it is because it makes it clear to the
>>>>>>>>>>>> user
>>>>>>> that
>>>>>>>>> a
>>>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>>>> intermediate
>>>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>>>
>>>>>>>>>>> Yes, that makes sense.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>>>>>>>>> discussed
>>>>>> most
>>>>>>>>> it
>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>>>>>>>> you
>>>>>> can
>>>>>>>>>>> reuse.
>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>> legal
>>>>>>> to
>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>> time?
>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>>>>>>> re-use
>>>>>> it
>>>>>>> as
>>>>>>>>> you
>>>>>>>>>>> can today.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>>>> pattern
>>>>>>> I
>>>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>>>> big
>>>>>>> fan
>>>>>>>>> of
>>>>>>>>>>>> the grouped.
>>>>>>>>>>>>
>>>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>>>> and
>>>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>>>> want to
>>>>>>>>> be
>>>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>>>> all
>>>>>>>>> depends
>>>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>>>> (probably
>>>>>>> 3)
>>>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>>>> KTable
>>>>>>>>> with
>>>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>>>> sessionwindowed
>>>>>>>>>>> for
>>>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>>>> Count,
>>>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>>>> builder:
>>>>>>>>>>>> // Count
>>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>>>>>>> // Windowed Count
>>>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>>>> // Session Count
>>>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>>>
>>>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> // Reduce
>>>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>>>
>>>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>>>
>>>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>>>> KTable<String, String> cogrouped =
>>>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>>>            .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>>>            .aggregate(initializer, aggregator,
>>>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> public class Count {
>>>>>>>>>>>>
>>>>>>>>>>>>        public static class Windowed extends Count {
>>>>>>>>>>>>            private Windows windows;
>>>>>>>>>>>>        }
>>>>>>>>>>>>        public static class SessionWindowed extends Count {
>>>>>>>>>>>>            private SessionWindows sessionWindows;
>>>>>>>>>>>>        }
>>>>>>>>>>>>
>>>>>>>>>>>>        public static Count count();
>>>>>>>>>>>>        public static Windowed windowed(Windows windows);
>>>>>>>>>>>>        public static SessionWindowed
>>>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>>>> sessionWindows);
>>>>>>>>>>>>
>>>>>>>>>>>>        // All withXXX(...) methods.
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>>>        public KTable<K, Long> count(Count count);
>>>>>>>>>>>>        public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>>>> count);
>>>>>>>>>>>>        public KTable<Windowed<K>, Long>
>>>>>>>>>>>> count(Count.SessionWindowed
>>>>>>>>> count);
>>>>>>>>>>>> …
>>>>>>>>>>>> }
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Kyle
>>>>>>>>>>>>
>>>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>>
>>>>>>>>>>>> I played the current proposal a bit with
>>>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>>>> tree/dsl-experiment <
>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>> ,
>>>>>>>>>>>> and here are my observations:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>>>
>>>>>>>>>>>>        "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>>>
>>>>>>>>>>>> than
>>>>>>>>>>>>
>>>>>>>>>>>>        "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>>>
>>>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically
>>>>>>>>>>>> though it
>>>>>> is
>>>>>>>>> not
>>>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>>>> natural.
>>>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>>>> )
>>>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user
>>>>>>>>>>>> supposed to
>>>>>>> pass
>>>>>>>>> in
>>>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>>>> return
>>>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>>>> supplier
>>>>>>> with
>>>>>>>>>>>> logging / metrics / etc? I think it would be more useful to
>>>>>>>>>>>> only
>>>>>>>>> require
>>>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>>>> caching /
>>>>>>>>>>>> logging through other APIs.
>>>>>>>>>>>>
>>>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit
>>>>>>>>>>>> suspicious to
>>>>>> me:
>>>>>>> we
>>>>>>>>>>> are
>>>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>>>> multiple
>>>>>>>>> time,
>>>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end.
>>>>>>>>>>>> Why
>>>>>> is
>>>>>>>>> that?
>>>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what
>>>>>>>>>>>> about
>>>>>>> joins?
>>>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>>>> topology
>>>>>>>>> so
>>>>>>>>>>> I
>>>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should
>>>>>>>>>>>> take
>>>>>> the
>>>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Below are minor ones:
>>>>>>>>>>>>
>>>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>>>
>>>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>>>> matthias@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I see your point about "when to add the processor to the
>>>>>> topology".
>>>>>>>>>>> That
>>>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to
>>>>>>>>>>>>> the
>>>>>>>>>>>> topology...
>>>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>>>> interface
>>>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>> aggregate()
>>>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>>> parameters
>>>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO,
>>>>>>>>>>>>> this
>>>>>>> might
>>>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>>>> I also think that mixing optional parameters with configs
>>>>>>>>>>>>>>> is a
>>>>>> bad
>>>>>>>>>>>> idea.
>>>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention
>>>>>>>>>>>>>>> it.
>>>>>> Hope
>>>>>>>>>>> to
>>>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>>>> config
>>>>>>>>>>>> here
>>>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>>>> already
>>>>>>>>>>> be
>>>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>>>> rename to
>>>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>>>> should
>>>>>>> find
>>>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>>>> close
>>>>>>>>>>>>>>> together. I would rather separate both more than less,
>>>>>>>>>>>>>>> ie, do
>>>>>> into
>>>>>>>>>>> the
>>>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>>>> The
>>>>>>>>>>>> initial
>>>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>>>> KTable
>>>>>>>>>>> which
>>>>>>>>>>>>> is
>>>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>>>> "build"
>>>>>>>>>>>> the
>>>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>>>> optional
>>>>>>>>>>>>>>> parameter for count() have to specified on the .grouped()
>>>>>>>>>>>>>>> call
>>>>>> -- 
>>>>>>>>>>> this
>>>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>>>> stream
>>>>>>>>>>> that
>>>>>>>>>>>>> you
>>>>>>>>>>>>>> will aggregate. So
>>>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>>>> etc
>>>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>>>> the
>>>>>>>>>>>> initial
>>>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>>>> .withXXX()
>>>>>>>>>>>> are
>>>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything
>>>>>>>>>>>>>>> why
>>>>>> this
>>>>>>>>>>>> wont'
>>>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> See above.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds
>>>>>>>>>>>>>>>> on the
>>>>>>>>>>> fluent
>>>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy
>>>>>>>>>>>>>>>> stuff
>>>>>> in
>>>>>>>>>>> the
>>>>>>>>>>>>> DSL,
>>>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>>>> things
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>>>> withLogConfig(...)
>>>>>>>>>>> (even
>>>>>>>>>>>>>>>> though it is below) as this is actually config and we
>>>>>>>>>>>>>>>> already
>>>>>>> have
>>>>>>>>>>> a
>>>>>>>>>>>>> way
>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we
>>>>>>>>>>>>>>>> could use
>>>>>> the
>>>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>>>> stands
>>>>>>> that
>>>>>>>>>>>> is
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>>>> the
>>>>>>>>>>>> default
>>>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>>>> to....
>>>>>>> If
>>>>>>>>>>>> you
>>>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it
>>>>>>>>>>>>>>>> to be
>>>>>>>>>>> logged
>>>>>>>>>>>>> etc,
>>>>>>>>>>>>>>>> you would currently need to implement that yourself.
>>>>>>>>>>>>>>>> Ideally
>>>>>> we
>>>>>>> can
>>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>>>> metrics,
>>>>>>>>>>>> etc. I
>>>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>>>> appropriate
>>>>>>>>>>>> on
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper =
>>>>>>>>>>>>>>>> null;
>>>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>>>            .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>>>            .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>>>            .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>>>            .withQueryableName("my-store")
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>>            .reduce(windowedReducer);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>>>            .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>>>            .aggregate(aggregator, init);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("name")
>>>>>>>>>>>>>>>>            .withCachingEnabled(false)
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withQueryableName("q")
>>>>>>>>>>>>>>>>            .withLoggingEnabled(false)
>>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>>            .withLogConfig(Collections.sin
>>>>>> gletonMap("segment.size",
>>>>>>>>>>>> "10"))
>>>>>>>>>>>>>>>>            .reduce(reducer);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>>>> code
>>>>>>>>>>> here:
>>>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>>>> return
>>>>>>>>>>> null.
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma
>>>>>>>>>>>>>>>> <is...@juma.me.uk>
>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons.
>>>>>>>>>>>>>>>>> And
>>>>>> both
>>>>>>>>>>> are
>>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of
>>>>>>>>>>>>>>>>> builder
>>>>>> or
>>>>>>>>>>> build
>>>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>>>> are a
>>>>>>>>>>>> little
>>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>>>> necessarily
>>>>>>>>>>>> knows
>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special
>>>>>>>>>>>>>>>>> method
>>>>>>>>>>>>> (`table()`
>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>>>> (`KTable`
>>>>>>>>>>> in
>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above,
>>>>>>>>>>>>>>>>> but
>>>>>> it
>>>>>>>>>>>> doesn't
>>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>>>> choices
>>>>>>>>>>>>> we've
>>>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>>>> API,
>>>>>>>>>>> i.e,
>>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>>>> becoming
>>>>>>>>>>> noisy
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more
>>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>> params.
>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>>>> caching
>>>>>>>>>>> off,
>>>>>>>>>>>> or
>>>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note
>>>>>>>>>>>>>>>>>> this can
>>>>>> be
>>>>>>>>>>> done
>>>>>>>>>>>>> now
>>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we
>>>>>>>>>>>>>>>>>> change the
>>>>>> DSL
>>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>>>> easily
>>>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>>>> providing
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>>>> method,
>>>>>>>>>>>> so
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what
>>>>>>>>>>>>>>>>>> are
>>>>>> you
>>>>>>> on
>>>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>>>> but
>>>>>>> the
>>>>>>>>>>>> same
>>>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>>>> many
>>>>>>> join
>>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>> -- 
>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>
>>>>> -- 
>>>>> -- Guozhang
>>>>>
>>>>
>>>
> 
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Matthias thanks,

Exactly what I was guessing.

I don't understand why custom stores in DSL? and I don't understand why 
we are not concidering a more generic config based appraoch?

StateStores in DSL => what I really think we are looking for PAPA => DSL 
=> PAPI  back and forth switcharoo capabilities.

Looking at the most overloaded that I can currently find "through()" 2 
of them come from the broken idea of "the user provides a name for the 
statestore for IQ" and custom statestores.
 From the beginning I said that's madness. That is the real disease we 
need to fix IMHO. To be honest I also don't understand why through with 
statestore is particularly usefull, second Unique Key maybe?

also providing Serdes by config is neat. wouldn't even need to go into 
the code then would also save a ton. (We have the defaults one in conf 
why not override the specific ones?)

Does this makes sense to people? what pieces should i outline with code 
(time is currently sparse :( but I can pull of some smaller examples i 
guess)

Best Jan





On 08.07.2017 01:23, Matthias J. Sax wrote:
> It's too issues we want to tackle
>
>   - too many overload (for some method we have already more than 10(
>   - improve custom store API
>
> -Matthias
>
>
> On 7/7/17 3:42 PM, Jan Filipiak wrote:
>> It makes me want to cry.
>>
>> why on earth is the DSL going to expose all its implementation details now?
>> especially being materialized or not.
>>
>> If we want to take usefull steps in that direction maybe we are looking
>> for a way to let the user switch back and forth between PAPI and DSL?
>>
>> A change as the proposed would not eliminate any of my pain points while
>> still being a heck of work migrating towards to.
>>
>> Since I am only following this from the point where Eno CC'ed it into
>> the users list:
>>
>> Can someone please rephrase for me what problem this is trying to solve?
>> I don't mean to be rude but It uses a problematic feature
>> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
>> us nowhere in making the configs more flexible, its just syntactic sugar.
>>
>> A low effort shoot like: lets add a properties to operations that would
>> otherwise become overloaded to heavy? Or pull the configs by some naming
>> schema
>> form the overall properties. Additionally to that we get rid of
>> StateStoreSuppliers in the DSL and have them also configured by said
>> properties.
>>
>> => way easier to migrate to, way less risk, way more flexible in the
>> future (different implementations of the same operation don't require
>> code change to configure)
>>
>> Line 184 makes especially no sense to me. what is a KTableKTable non
>> materialized join anyways?
>>
>> Hope we can discuss more on this.
>>
>>
>>
>> On 07.07.2017 17:23, Guozhang Wang wrote:
>>> I messed the indentation on github code repos; this would be easier to
>>> read:
>>>
>>> https://codeshare.io/GLWW8K
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:
>>>
>>>> Hi Damian / Kyle,
>>>>
>>>> I think I agree with you guys about the pros / cons of using the builder
>>>> pattern v.s. using some "secondary classes". And I'm thinking if we can
>>>> take a "mid" manner between these two. I spent some time with a slight
>>>> different approach from Damian's current proposal:
>>>>
>>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>>
>>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>>
>>>> The key idea is to tolerate the final "table()" or "stream()"
>>>> function to
>>>> "upgrade" from the secondary classes to the first citizen classes, while
>>>> having all the specs inside this function. Also this proposal
>>>> includes some
>>>> other refactoring that people have been discussed about for the
>>>> builder to
>>>> reduce the overloaded functions as well. WDYT?
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>>>>
>>>>> Hi Jan,
>>>>>
>>>>> Thanks very much for the input.
>>>>>
>>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Damian,
>>>>>>
>>>>>> I do see your point of something needs to change. But I fully agree
>>>>>> with
>>>>>> Gouzhang when he says.
>>>>>> ---
>>>>>>
>>>>>> But since this is a incompatibility change, and we are going to remove
>>>>> the
>>>>>> compatibility annotations soon it means we only have one chance and we
>>>>>> really have to make it right.
>>>>>> ----
>>>>>>
>>>>>>
>>>>> I think we all agree on this one! Hence the discussion.
>>>>>
>>>>>
>>>>>> I fear all suggestions do not go far enough to become something that
>>>>> will
>>>>>> carry on for very much longer.
>>>>>> I am currently working on KAFKA-3705 and try to find the most easy way
>>>>> for
>>>>>> the user to give me all the required functionality. The easiest
>>>>> interface I
>>>>>> could come up so far can be looked at here.
>>>>>>
>>>>>>
>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>> And its already horribly complicated. I am currently unable to find the
>>>>>> right abstraction level to have everything falling into place
>>>>> naturally. To
>>>>>> be honest I already think introducing
>>>>>>
>>>>>>
>>>>> To be fair that is not a particularly easy problem to solve!
>>>>>
>>>>>
>>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>>> was unideal and makes everything a mess.
>>>>> I'm not sure i agree that it makes everything a mess, but It could have
>>>>> been done differently.
>>>>>
>>>>> The JoinType:Whatever is also not really flexible. 2 things come to my
>>>>> mind:
>>>>>> 1. I don't think we should rule out config based decisions say configs
>>>>> like
>>>>>>           streams.$applicationID.joins.$joinname.conf = value
>>>>>>
>>>>> Is this just for config? Or are you suggesting that we could somehow
>>>>> "code"
>>>>> the join in a config file?
>>>>>
>>>>>
>>>>>> This can allow for tremendous changes without single API change and
>>>>>> IMO
>>>>> it
>>>>>> was not considered enough yet.
>>>>>>
>>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>>>>>> example can be used to implement different join types as the user
>>>>> wishes.
>>>>> Do you have an example of how this might look?
>>>>>
>>>>>
>>>>>> As Gouzhang said: stopping to break users is very important.
>>>>> Of course. We want to make it as easy as possible for people to use
>>>>> streams.
>>>>>
>>>>>
>>>>> especially with this changes + All the plans I sadly only have in my
>>>>> head
>>>>>> but hopefully the first link can give a glimpse.
>>>>>>
>>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>>> exactly
>>>>>> we are talking about. I would argue to go a bit slower and more
>>>>> carefull on
>>>>>> this one. At some point we need to get it right. Peeking over to the
>>>>> hadoop
>>>>>> guys with their hughe userbase. Config files really work well for
>>>>>> them.
>>>>>>
>>>>>> Best Jan
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>>> Thanks Matthias
>>>>>>>
>>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>>>>>> wrote:
>>>>>>>> I am just catching up on this thread, so sorry for the long email in
>>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>>> always a
>>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>>> maybe
>>>>>>>> it helps other to get new ideas :)
>>>>>>>>
>>>>>>>>
>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>>> aggregate()
>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>> parameters
>>>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>> might
>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> I see what you are saying, but the grouped stream is effectively a
>>>>>> no-op
>>>>>>>>> until you call one of the aggregate/count/reduce etc functions. So
>>>>> the
>>>>>>>>> optional params are ones that are applicable to any of the
>>>>> operations
>>>>>> you
>>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>>> required/specific to that function.
>>>>>>>>>
>>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>>> If we
>>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>>
>>>>>>>> .groupBy().count().withXX().build()
>>>>>>>>
>>>>>>>> (with a better name for build() though)
>>>>>>>>
>>>>>>>>
>>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>>> apply
>>>>>>> too all the aggregate functions. The terminal call being the actual
>>>>> type
>>>>>> of
>>>>>>> aggregation you want to do. I personally find this more natural than
>>>>>>> groupBy().count().withBlah().build()
>>>>>>>
>>>>>>>
>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>>> sessionWindowed(). We should have one top-level `Windows` interface
>>>>> that
>>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>>> single
>>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>>> separation of `SessionWindows` in the first place, and this seems to
>>>>> be
>>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>>> introduced session windows)
>>>>>>>>
>>>>>>> Yes - true we should look into that.
>>>>>>>
>>>>>>>
>>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>>> thus we
>>>>>>>> might also want to use windowBy() (instead of windowed()). Not sure
>>>>> how
>>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>>
>>>>>>>>
>>>>>>> Makes sense
>>>>>>>
>>>>>>>
>>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>>> think,
>>>>>>>> defining an inner/left/outer join is not an optional argument but a
>>>>>>>> first class concept and should have a proper representation in the
>>>>> API
>>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>>
>>>>>>>>
>>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>>> what
>>>>>> we
>>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> About the two join API proposals, the second one has too much boiler
>>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>>> one
>>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>>> operator call, should have one parameter per mandatory argument but
>>>>> your
>>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>>> call.
>>>>>>>> This is far from intuitive IMHO.
>>>>>>>>
>>>>>>>>
>>>>>>> This is the builder pattern, you only need one param as the builder
>>>>> has
>>>>>>> captured all of the required and optional arguments.
>>>>>>>
>>>>>>>
>>>>>>>> The first join proposal also seems to align better with the pattern
>>>>>>>> suggested for aggregations and having the same pattern for all
>>>>> operators
>>>>>>>> is important (as you stated already).
>>>>>>>>
>>>>>>>>
>>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>>> the
>>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>>
>>>>>>>
>>>>>>>> Coming back to the config vs optional parameter. What about having a
>>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>>
>>>>>>>>
>>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>>> thing
>>>>>> that
>>>>>>> is really config.
>>>>>>>
>>>>>>>
>>>>>>>> This also raises the question if until() is a windows property?
>>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>>> should
>>>>>>>> not not have it's own method.
>>>>>>>>
>>>>>>>>
>>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>>> to be
>>>>>>> potentially different for every window operation you do in a streams
>>>>> app.
>>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>>
>>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>>     groupedStream.counting()
>>>>>>>>>                     .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>                     .table();
>>>>>>>> This is an interesting idea, and it remind my on some feedback about
>>>>> "I
>>>>>>>> wanted to count a stream, but there was no count() method -- I first
>>>>>>>> needed to figure out, that I need to group the stream first to be
>>>>> able
>>>>>>>> to count it. It does make sense in hindsight but was not obvious in
>>>>> the
>>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>>> following:
>>>>>>>>
>>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>>
>>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative here,
>>>>> as
>>>>>>>> it comes after the count()
>>>>>>>>
>>>>>>>> This would be more consistent than your proposal (that has grouping
>>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>>> enrich
>>>>>>>> the API with a some syntactic sugar like `stream.count().table()` to
>>>>> get
>>>>>>>> the overall count of all records (this would obviously not scale,
>>>>> but we
>>>>>>>> could support it -- if not now, maybe later).
>>>>>>>>
>>>>>>>>
>>>>>>> I guess i'd prefer
>>>>>>> stream.groupBy().windowBy().count()
>>>>>>> stream.groupBy().windowBy().reduce()
>>>>>>> stream.groupBy().count()
>>>>>>>
>>>>>>> As i said above, everything that happens before the final aggregate
>>>>> call
>>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>>> things
>>>>>>> ahead of the final aggregate call.
>>>>>>>
>>>>>>>
>>>>>>>> Last about builder pattern. I am convinced that we need some
>>>>> "terminal"
>>>>>>>> operator/method that tells us when to add the processor to the
>>>>> topology.
>>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>>> alien to
>>>>>>>> me (see my argument about the second join proposal). Using .stream()
>>>>> /
>>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>>> generic
>>>>>>>> name that we can use in all places like build() or apply() might
>>>>> also be
>>>>>>>> an option.
>>>>>>>>
>>>>>>>>
>>>>>>> Sure, a generic name might be ok.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>>> Thanks Kyle.
>>>>>>>>>
>>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>>> winkelman.kyle@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi Damian,
>>>>>>>>>>
>>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>>> discussed
>>>>>>>> most
>>>>>>>>>> it
>>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>> you
>>>>>> can
>>>>>>>>>> reuse.
>>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>> legal
>>>>>>>> to
>>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>> time?
>>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>> re-use it
>>>>>>>> as
>>>>>>>>>> you
>>>>>>>>>>>> can today.
>>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I
>>>>> see
>>>>>>>> it
>>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>>> KTables,
>>>>>>>> because
>>>>>>>>>> most of this fluent api would continue returning this (this being
>>>>> the
>>>>>>>>>> builder object currently being manipulated).
>>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>>> KTables
>>>>>>>> and
>>>>>>>>>> you use it in two different ways then its possible you make
>>>>>> conflicting
>>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> No necessarily true. It could return a new instance of the builder,
>>>>>> i.e.,
>>>>>>>>> the builders being immutable. So if you held a reference to the
>>>>> builder
>>>>>>>> it
>>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>>> kStream.grouped();
>>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>>
>>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going to do
>>>>> it
>>>>>>>>>> eventually…
>>>>>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>>>>>> would
>>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>>> work
>>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>>> are
>>>>>>>>>> building, but this approach has its own problem because now we
>>>>> have to
>>>>>>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>>>>>> processors.
>>>>>>>>>>
>>>>>>>>>> The point of this long winded example is that we always have to be
>>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>>> user
>>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>>
>>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>>> could
>>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>>> // Count
>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>
>>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>
>>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>>>>>> Count
>>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>>> store
>>>>>>>> named
>>>>>>>>>> “my-store” already exists.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Kyle
>>>>>>>>>>
>>>>>>>>>> From: Damian Guy
>>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>
>>>>>>>>>> Hi Kyle,
>>>>>>>>>>
>>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>>
>>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>>> winkelman.kyle@gmail.com
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>>> against
>>>>>>>>>>> it. The reason I like it is because it makes it clear to the user
>>>>>> that
>>>>>>>> a
>>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>>> intermediate
>>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>>
>>>>>>>>>> Yes, that makes sense.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> When trying to program in the fluent API that has been discussed
>>>>> most
>>>>>>>> it
>>>>>>>>>>> feels difficult to know when you will actually get an object you
>>>>> can
>>>>>>>>>> reuse.
>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>> legal
>>>>>> to
>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>> time?
>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
>>>>> it
>>>>>> as
>>>>>>>> you
>>>>>>>>>> can today.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>>> pattern
>>>>>> I
>>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>>> big
>>>>>> fan
>>>>>>>> of
>>>>>>>>>>> the grouped.
>>>>>>>>>>>
>>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>>> and
>>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>>> want to
>>>>>>>> be
>>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>>> all
>>>>>>>> depends
>>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>>> (probably
>>>>>> 3)
>>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>>> KTable
>>>>>>>> with
>>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>>> sessionwindowed
>>>>>>>>>> for
>>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>>> Count,
>>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>>> builder:
>>>>>>>>>>> // Count
>>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>>
>>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>>>> // Windowed Count
>>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>>
>>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>>> // Session Count
>>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>>
>>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>> // Reduce
>>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>>
>>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>>
>>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>>> KTable<String, String> cogrouped =
>>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>>            .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>>            .aggregate(initializer, aggregator,
>>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> public class Count {
>>>>>>>>>>>
>>>>>>>>>>>        public static class Windowed extends Count {
>>>>>>>>>>>            private Windows windows;
>>>>>>>>>>>        }
>>>>>>>>>>>        public static class SessionWindowed extends Count {
>>>>>>>>>>>            private SessionWindows sessionWindows;
>>>>>>>>>>>        }
>>>>>>>>>>>
>>>>>>>>>>>        public static Count count();
>>>>>>>>>>>        public static Windowed windowed(Windows windows);
>>>>>>>>>>>        public static SessionWindowed
>>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>>> sessionWindows);
>>>>>>>>>>>
>>>>>>>>>>>        // All withXXX(...) methods.
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>>        public KTable<K, Long> count(Count count);
>>>>>>>>>>>        public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>>> count);
>>>>>>>>>>>        public KTable<Windowed<K>, Long>
>>>>>>>>>>> count(Count.SessionWindowed
>>>>>>>> count);
>>>>>>>>>>> …
>>>>>>>>>>> }
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Kyle
>>>>>>>>>>>
>>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>>
>>>>>>>>>>> I played the current proposal a bit with
>>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>>> tree/dsl-experiment <
>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>> ,
>>>>>>>>>>> and here are my observations:
>>>>>>>>>>>
>>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>>
>>>>>>>>>>>        "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>>
>>>>>>>>>>> than
>>>>>>>>>>>
>>>>>>>>>>>        "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>>
>>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it
>>>>> is
>>>>>>>> not
>>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>>> natural.
>>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>>> )
>>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>>>>>> pass
>>>>>>>> in
>>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>>> return
>>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>>> supplier
>>>>>> with
>>>>>>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>>>>>> require
>>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>>> caching /
>>>>>>>>>>> logging through other APIs.
>>>>>>>>>>>
>>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>>>>> me:
>>>>>> we
>>>>>>>>>> are
>>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>>> multiple
>>>>>>>> time,
>>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why
>>>>> is
>>>>>>>> that?
>>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>>>>>> joins?
>>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>>> topology
>>>>>>>> so
>>>>>>>>>> I
>>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>>>>> the
>>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Below are minor ones:
>>>>>>>>>>>
>>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>>
>>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>>> matthias@confluent.io>
>>>>>>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I see your point about "when to add the processor to the
>>>>> topology".
>>>>>>>>>> That
>>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>>>>>> topology...
>>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>>> interface
>>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> However, I don't understand your argument about putting
>>>>> aggregate()
>>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>> parameters
>>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>> might
>>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>>> I also think that mixing optional parameters with configs is a
>>>>> bad
>>>>>>>>>>> idea.
>>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention it.
>>>>> Hope
>>>>>>>>>> to
>>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>>> config
>>>>>>>>>>> here
>>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>>> already
>>>>>>>>>> be
>>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>>> rename to
>>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>>> should
>>>>>> find
>>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>>> close
>>>>>>>>>>>>>> together. I would rather separate both more than less, ie, do
>>>>> into
>>>>>>>>>> the
>>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>>> The
>>>>>>>>>>> initial
>>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>>
>>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>>> KTable
>>>>>>>>>> which
>>>>>>>>>>>> is
>>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>>> "build"
>>>>>>>>>>> the
>>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>>> optional
>>>>>>>>>>>>>> parameter for count() have to specified on the .grouped() call
>>>>> -- 
>>>>>>>>>> this
>>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>>> stream
>>>>>>>>>> that
>>>>>>>>>>>> you
>>>>>>>>>>>>> will aggregate. So
>>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>>> etc
>>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>>> the
>>>>>>>>>>> initial
>>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>>> .withXXX()
>>>>>>>>>>> are
>>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything why
>>>>> this
>>>>>>>>>>> wont'
>>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>> See above.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>>>>>>>> fluent
>>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>>>>> in
>>>>>>>>>> the
>>>>>>>>>>>> DSL,
>>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>>> things
>>>>>>>>>>> that
>>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>>> withLogConfig(...)
>>>>>>>>>> (even
>>>>>>>>>>>>>>> though it is below) as this is actually config and we already
>>>>>> have
>>>>>>>>>> a
>>>>>>>>>>>> way
>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>>>>> the
>>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>>> stands
>>>>>> that
>>>>>>>>>>> is
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>>> the
>>>>>>>>>>> default
>>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>>> to....
>>>>>> If
>>>>>>>>>>> you
>>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>>>>>>>> logged
>>>>>>>>>>>> etc,
>>>>>>>>>>>>>>> you would currently need to implement that yourself. Ideally
>>>>> we
>>>>>> can
>>>>>>>>>>>>>> provide
>>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>>> metrics,
>>>>>>>>>>> etc. I
>>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>>> appropriate
>>>>>>>>>>> on
>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>>            .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>>            .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>>            .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>>            .withQueryableName("my-store")
>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withQueryableName("my-window-store")
>>>>>>>>>>>>>>>            .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>>            .reduce(windowedReducer);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>>            .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>>            .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>>            .aggregate(aggregator, init);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withQueryableName("name")
>>>>>>>>>>>>>>>            .withCachingEnabled(false)
>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withQueryableName("q")
>>>>>>>>>>>>>>>            .withLoggingEnabled(false)
>>>>>>>>>>>>>>>            .count();
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>>            .withLogConfig(Collections.sin
>>>>> gletonMap("segment.size",
>>>>>>>>>>> "10"))
>>>>>>>>>>>>>>>            .reduce(reducer);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>>> code
>>>>>>>>>> here:
>>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>>> return
>>>>>>>>>> null.
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>>>>> both
>>>>>>>>>> are
>>>>>>>>>>>>>> better
>>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of builder
>>>>> or
>>>>>>>>>> build
>>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>>> are a
>>>>>>>>>>> little
>>>>>>>>>>>>>> less
>>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>>> necessarily
>>>>>>>>>>> knows
>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>>>>>>>> (`table()`
>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>>> (`KTable`
>>>>>>>>>> in
>>>>>>>>>>>> this
>>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above, but
>>>>> it
>>>>>>>>>>> doesn't
>>>>>>>>>>>>>> read
>>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>>> damian.guy@gmail.com
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>>> choices
>>>>>>>>>>>> we've
>>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>>> operations
>>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>>> API,
>>>>>>>>>> i.e,
>>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>>> becoming
>>>>>>>>>> noisy
>>>>>>>>>>>> and
>>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>>>>>>>> params.
>>>>>>>>>>> In
>>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>>> caching
>>>>>>>>>> off,
>>>>>>>>>>> or
>>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note this can
>>>>> be
>>>>>>>>>> done
>>>>>>>>>>>> now
>>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we change the
>>>>> DSL
>>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>>> easily
>>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>>> providing
>>>>>>>>>> the
>>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>>       .withStoreName("name")
>>>>>>>>>>>>>>>>>       .withCachingEnabled(false)
>>>>>>>>>>>>>>>>>       .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>>       .table()
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>>> method,
>>>>>>>>>>> so
>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>>>>> you
>>>>>> on
>>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>>> but
>>>>>> the
>>>>>>>>>>> same
>>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>>> many
>>>>>> join
>>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>
>>>>>>>>>>> -- 
>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>
>>>>>>>>>>>
>>>>
>>>> -- 
>>>> -- Guozhang
>>>>
>>>
>>



Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
It's too issues we want to tackle

 - too many overload (for some method we have already more than 10(
 - improve custom store API

-Matthias


On 7/7/17 3:42 PM, Jan Filipiak wrote:
> It makes me want to cry.
> 
> why on earth is the DSL going to expose all its implementation details now?
> especially being materialized or not.
> 
> If we want to take usefull steps in that direction maybe we are looking
> for a way to let the user switch back and forth between PAPI and DSL?
> 
> A change as the proposed would not eliminate any of my pain points while
> still being a heck of work migrating towards to.
> 
> Since I am only following this from the point where Eno CC'ed it into
> the users list:
> 
> Can someone please rephrase for me what problem this is trying to solve?
> I don't mean to be rude but It uses a problematic feature
> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
> us nowhere in making the configs more flexible, its just syntactic sugar.
> 
> A low effort shoot like: lets add a properties to operations that would
> otherwise become overloaded to heavy? Or pull the configs by some naming
> schema
> form the overall properties. Additionally to that we get rid of
> StateStoreSuppliers in the DSL and have them also configured by said
> properties.
> 
> => way easier to migrate to, way less risk, way more flexible in the
> future (different implementations of the same operation don't require
> code change to configure)
> 
> Line 184 makes especially no sense to me. what is a KTableKTable non
> materialized join anyways?
> 
> Hope we can discuss more on this.
> 
> 
> 
> On 07.07.2017 17:23, Guozhang Wang wrote:
>> I messed the indentation on github code repos; this would be easier to
>> read:
>>
>> https://codeshare.io/GLWW8K
>>
>>
>> Guozhang
>>
>>
>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Hi Damian / Kyle,
>>>
>>> I think I agree with you guys about the pros / cons of using the builder
>>> pattern v.s. using some "secondary classes". And I'm thinking if we can
>>> take a "mid" manner between these two. I spent some time with a slight
>>> different approach from Damian's current proposal:
>>>
>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>
>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>
>>> The key idea is to tolerate the final "table()" or "stream()"
>>> function to
>>> "upgrade" from the secondary classes to the first citizen classes, while
>>> having all the specs inside this function. Also this proposal
>>> includes some
>>> other refactoring that people have been discussed about for the
>>> builder to
>>> reduce the overloaded functions as well. WDYT?
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>>>
>>>> Hi Jan,
>>>>
>>>> Thanks very much for the input.
>>>>
>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>> Hi Damian,
>>>>>
>>>>> I do see your point of something needs to change. But I fully agree
>>>>> with
>>>>> Gouzhang when he says.
>>>>> ---
>>>>>
>>>>> But since this is a incompatibility change, and we are going to remove
>>>> the
>>>>> compatibility annotations soon it means we only have one chance and we
>>>>> really have to make it right.
>>>>> ----
>>>>>
>>>>>
>>>> I think we all agree on this one! Hence the discussion.
>>>>
>>>>
>>>>> I fear all suggestions do not go far enough to become something that
>>>> will
>>>>> carry on for very much longer.
>>>>> I am currently working on KAFKA-3705 and try to find the most easy way
>>>> for
>>>>> the user to give me all the required functionality. The easiest
>>>> interface I
>>>>> could come up so far can be looked at here.
>>>>>
>>>>>
>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>>
>>>> And its already horribly complicated. I am currently unable to find the
>>>>> right abstraction level to have everything falling into place
>>>> naturally. To
>>>>> be honest I already think introducing
>>>>>
>>>>>
>>>> To be fair that is not a particularly easy problem to solve!
>>>>
>>>>
>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>> was unideal and makes everything a mess.
>>>>
>>>> I'm not sure i agree that it makes everything a mess, but It could have
>>>> been done differently.
>>>>
>>>> The JoinType:Whatever is also not really flexible. 2 things come to my
>>>> mind:
>>>>> 1. I don't think we should rule out config based decisions say configs
>>>> like
>>>>>          streams.$applicationID.joins.$joinname.conf = value
>>>>>
>>>> Is this just for config? Or are you suggesting that we could somehow
>>>> "code"
>>>> the join in a config file?
>>>>
>>>>
>>>>> This can allow for tremendous changes without single API change and
>>>>> IMO
>>>> it
>>>>> was not considered enough yet.
>>>>>
>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>>>>> example can be used to implement different join types as the user
>>>> wishes.
>>>> Do you have an example of how this might look?
>>>>
>>>>
>>>>> As Gouzhang said: stopping to break users is very important.
>>>>
>>>> Of course. We want to make it as easy as possible for people to use
>>>> streams.
>>>>
>>>>
>>>> especially with this changes + All the plans I sadly only have in my
>>>> head
>>>>> but hopefully the first link can give a glimpse.
>>>>>
>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>> exactly
>>>>> we are talking about. I would argue to go a bit slower and more
>>>> carefull on
>>>>> this one. At some point we need to get it right. Peeking over to the
>>>> hadoop
>>>>> guys with their hughe userbase. Config files really work well for
>>>>> them.
>>>>>
>>>>> Best Jan
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>> Thanks Matthias
>>>>>>
>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>>> I am just catching up on this thread, so sorry for the long email in
>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>> always a
>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>> maybe
>>>>>>> it helps other to get new ideas :)
>>>>>>>
>>>>>>>
>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>> aggregate()
>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>> parameters
>>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>> might
>>>>>>>>> be quite confusion for developers.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> I see what you are saying, but the grouped stream is effectively a
>>>>> no-op
>>>>>>>> until you call one of the aggregate/count/reduce etc functions. So
>>>> the
>>>>>>>> optional params are ones that are applicable to any of the
>>>> operations
>>>>> you
>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>> required/specific to that function.
>>>>>>>>
>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>> If we
>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>
>>>>>>> .groupBy().count().withXX().build()
>>>>>>>
>>>>>>> (with a better name for build() though)
>>>>>>>
>>>>>>>
>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>> apply
>>>>>> too all the aggregate functions. The terminal call being the actual
>>>> type
>>>>> of
>>>>>> aggregation you want to do. I personally find this more natural than
>>>>>> groupBy().count().withBlah().build()
>>>>>>
>>>>>>
>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>> sessionWindowed(). We should have one top-level `Windows` interface
>>>> that
>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>> single
>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>> separation of `SessionWindows` in the first place, and this seems to
>>>> be
>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>> introduced session windows)
>>>>>>>
>>>>>> Yes - true we should look into that.
>>>>>>
>>>>>>
>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>> thus we
>>>>>>> might also want to use windowBy() (instead of windowed()). Not sure
>>>> how
>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>
>>>>>>>
>>>>>> Makes sense
>>>>>>
>>>>>>
>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>> think,
>>>>>>> defining an inner/left/outer join is not an optional argument but a
>>>>>>> first class concept and should have a proper representation in the
>>>> API
>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>
>>>>>>>
>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>> what
>>>>> we
>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>
>>>>>>
>>>>>>
>>>>>>> About the two join API proposals, the second one has too much boiler
>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>> one
>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>> operator call, should have one parameter per mandatory argument but
>>>> your
>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>> call.
>>>>>>> This is far from intuitive IMHO.
>>>>>>>
>>>>>>>
>>>>>> This is the builder pattern, you only need one param as the builder
>>>> has
>>>>>> captured all of the required and optional arguments.
>>>>>>
>>>>>>
>>>>>>> The first join proposal also seems to align better with the pattern
>>>>>>> suggested for aggregations and having the same pattern for all
>>>> operators
>>>>>>> is important (as you stated already).
>>>>>>>
>>>>>>>
>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>> the
>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>
>>>>>>
>>>>>>> Coming back to the config vs optional parameter. What about having a
>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>
>>>>>>>
>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>> thing
>>>>> that
>>>>>> is really config.
>>>>>>
>>>>>>
>>>>>>> This also raises the question if until() is a windows property?
>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>> should
>>>>>>> not not have it's own method.
>>>>>>>
>>>>>>>
>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>> to be
>>>>>> potentially different for every window operation you do in a streams
>>>> app.
>>>>>>
>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>
>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>    groupedStream.counting()
>>>>>>>>                    .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>                    .table();
>>>>>>> This is an interesting idea, and it remind my on some feedback about
>>>> "I
>>>>>>> wanted to count a stream, but there was no count() method -- I first
>>>>>>> needed to figure out, that I need to group the stream first to be
>>>> able
>>>>>>> to count it. It does make sense in hindsight but was not obvious in
>>>> the
>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>> following:
>>>>>>>
>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>
>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative here,
>>>> as
>>>>>>> it comes after the count()
>>>>>>>
>>>>>>> This would be more consistent than your proposal (that has grouping
>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>> enrich
>>>>>>> the API with a some syntactic sugar like `stream.count().table()` to
>>>> get
>>>>>>> the overall count of all records (this would obviously not scale,
>>>> but we
>>>>>>> could support it -- if not now, maybe later).
>>>>>>>
>>>>>>>
>>>>>> I guess i'd prefer
>>>>>> stream.groupBy().windowBy().count()
>>>>>> stream.groupBy().windowBy().reduce()
>>>>>> stream.groupBy().count()
>>>>>>
>>>>>> As i said above, everything that happens before the final aggregate
>>>> call
>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>> things
>>>>>> ahead of the final aggregate call.
>>>>>>
>>>>>>
>>>>>>> Last about builder pattern. I am convinced that we need some
>>>> "terminal"
>>>>>>> operator/method that tells us when to add the processor to the
>>>> topology.
>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>> alien to
>>>>>>> me (see my argument about the second join proposal). Using .stream()
>>>> /
>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>> generic
>>>>>>> name that we can use in all places like build() or apply() might
>>>> also be
>>>>>>> an option.
>>>>>>>
>>>>>>>
>>>>>> Sure, a generic name might be ok.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>> Thanks Kyle.
>>>>>>>>
>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>> winkelman.kyle@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Damian,
>>>>>>>>>
>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>> discussed
>>>>>>> most
>>>>>>>>> it
>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>> you
>>>>> can
>>>>>>>>> reuse.
>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>> legal
>>>>>>> to
>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>> time?
>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>> re-use it
>>>>>>> as
>>>>>>>>> you
>>>>>>>>>>> can today.
>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I
>>>> see
>>>>>>> it
>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>> KTables,
>>>>>>> because
>>>>>>>>> most of this fluent api would continue returning this (this being
>>>> the
>>>>>>>>> builder object currently being manipulated).
>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>> KTables
>>>>>>> and
>>>>>>>>> you use it in two different ways then its possible you make
>>>>> conflicting
>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> No necessarily true. It could return a new instance of the builder,
>>>>> i.e.,
>>>>>>>> the builders being immutable. So if you held a reference to the
>>>> builder
>>>>>>> it
>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>
>>>>>>>>
>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>> kStream.grouped();
>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>
>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going to do
>>>> it
>>>>>>>>> eventually…
>>>>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>>>>> would
>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>> work
>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>> are
>>>>>>>>> building, but this approach has its own problem because now we
>>>> have to
>>>>>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>>>>> processors.
>>>>>>>>>
>>>>>>>>> The point of this long winded example is that we always have to be
>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>> user
>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>
>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>
>>>>>>>>
>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>> could
>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>
>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>>>>> Count
>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>> store
>>>>>>> named
>>>>>>>>> “my-store” already exists.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Kyle
>>>>>>>>>
>>>>>>>>> From: Damian Guy
>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>
>>>>>>>>> Hi Kyle,
>>>>>>>>>
>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>
>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>> winkelman.kyle@gmail.com
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>> against
>>>>>>>>>> it. The reason I like it is because it makes it clear to the user
>>>>> that
>>>>>>> a
>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>> intermediate
>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>
>>>>>>>>> Yes, that makes sense.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> When trying to program in the fluent API that has been discussed
>>>> most
>>>>>>> it
>>>>>>>>>> feels difficult to know when you will actually get an object you
>>>> can
>>>>>>>>> reuse.
>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>> legal
>>>>> to
>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>> time?
>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
>>>> it
>>>>> as
>>>>>>> you
>>>>>>>>> can today.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>> pattern
>>>>> I
>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>> big
>>>>> fan
>>>>>>> of
>>>>>>>>>> the grouped.
>>>>>>>>>>
>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>> and
>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>> want to
>>>>>>> be
>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>> all
>>>>>>> depends
>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>> (probably
>>>>> 3)
>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>> KTable
>>>>>>> with
>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>> sessionwindowed
>>>>>>>>> for
>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>> Count,
>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>> builder:
>>>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>>> // Windowed Count
>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>
>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>> // Session Count
>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>
>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> // Reduce
>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>
>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>> KTable<String, String> cogrouped =
>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>           .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>           .aggregate(initializer, aggregator,
>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> public class Count {
>>>>>>>>>>
>>>>>>>>>>       public static class Windowed extends Count {
>>>>>>>>>>           private Windows windows;
>>>>>>>>>>       }
>>>>>>>>>>       public static class SessionWindowed extends Count {
>>>>>>>>>>           private SessionWindows sessionWindows;
>>>>>>>>>>       }
>>>>>>>>>>
>>>>>>>>>>       public static Count count();
>>>>>>>>>>       public static Windowed windowed(Windows windows);
>>>>>>>>>>       public static SessionWindowed
>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>> sessionWindows);
>>>>>>>>>>
>>>>>>>>>>       // All withXXX(...) methods.
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>       public KTable<K, Long> count(Count count);
>>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>> count);
>>>>>>>>>>       public KTable<Windowed<K>, Long>
>>>>>>>>>> count(Count.SessionWindowed
>>>>>>> count);
>>>>>>>>>> …
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Kyle
>>>>>>>>>>
>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>
>>>>>>>>>> I played the current proposal a bit with
>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>> tree/dsl-experiment <
>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>> ,
>>>>>>>>>> and here are my observations:
>>>>>>>>>>
>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>
>>>>>>>>>>       "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>
>>>>>>>>>> than
>>>>>>>>>>
>>>>>>>>>>       "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>
>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it
>>>> is
>>>>>>> not
>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>> natural.
>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>
>>>>>>>>>>
>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>> )
>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>>>>> pass
>>>>>>> in
>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>> return
>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>> supplier
>>>>> with
>>>>>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>>>>> require
>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>> caching /
>>>>>>>>>> logging through other APIs.
>>>>>>>>>>
>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>>>> me:
>>>>> we
>>>>>>>>> are
>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>> multiple
>>>>>>> time,
>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why
>>>> is
>>>>>>> that?
>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>>>>> joins?
>>>>>>>>>>
>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>> topology
>>>>>>> so
>>>>>>>>> I
>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>>>> the
>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Below are minor ones:
>>>>>>>>>>
>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>
>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>> matthias@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> I see your point about "when to add the processor to the
>>>> topology".
>>>>>>>>> That
>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>>>>> topology...
>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>> interface
>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> However, I don't understand your argument about putting
>>>> aggregate()
>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>> parameters
>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>> might
>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>> I also think that mixing optional parameters with configs is a
>>>> bad
>>>>>>>>>> idea.
>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention it.
>>>> Hope
>>>>>>>>> to
>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>> config
>>>>>>>>>> here
>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>> already
>>>>>>>>> be
>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>> rename to
>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>> should
>>>>> find
>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>> close
>>>>>>>>>>>>> together. I would rather separate both more than less, ie, do
>>>> into
>>>>>>>>> the
>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>> The
>>>>>>>>>> initial
>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>
>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>> KTable
>>>>>>>>> which
>>>>>>>>>>> is
>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>> "build"
>>>>>>>>>> the
>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>> optional
>>>>>>>>>>>>> parameter for count() have to specified on the .grouped() call
>>>> -- 
>>>>>>>>> this
>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>> stream
>>>>>>>>> that
>>>>>>>>>>> you
>>>>>>>>>>>> will aggregate. So
>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>> etc
>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>> the
>>>>>>>>>> initial
>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>
>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>
>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>> .withXXX()
>>>>>>>>>> are
>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything why
>>>> this
>>>>>>>>>> wont'
>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> See above.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>>>>>>> fluent
>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>>>> in
>>>>>>>>> the
>>>>>>>>>>> DSL,
>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>> things
>>>>>>>>>> that
>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>> withLogConfig(...)
>>>>>>>>> (even
>>>>>>>>>>>>>> though it is below) as this is actually config and we already
>>>>> have
>>>>>>>>> a
>>>>>>>>>>> way
>>>>>>>>>>>>> of
>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>>>> the
>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>> stands
>>>>> that
>>>>>>>>>> is
>>>>>>>>>>> a
>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>> the
>>>>>>>>>> default
>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>> to....
>>>>> If
>>>>>>>>>> you
>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>>>>>>> logged
>>>>>>>>>>> etc,
>>>>>>>>>>>>>> you would currently need to implement that yourself. Ideally
>>>> we
>>>>> can
>>>>>>>>>>>>> provide
>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>> metrics,
>>>>>>>>>> etc. I
>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>> appropriate
>>>>>>>>>> on
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>           .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>           .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>           .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>           .withQueryableName("my-store")
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>           .reduce(windowedReducer);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>           .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>> stream.grouped()
>>>>>>>>>>>>>>           .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>           .aggregate(aggregator, init);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("name")
>>>>>>>>>>>>>>           .withCachingEnabled(false)
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("q")
>>>>>>>>>>>>>>           .withLoggingEnabled(false)
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withLogConfig(Collections.sin
>>>> gletonMap("segment.size",
>>>>>>>>>> "10"))
>>>>>>>>>>>>>>           .reduce(reducer);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>> code
>>>>>>>>> here:
>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>> return
>>>>>>>>> null.
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>>>> both
>>>>>>>>> are
>>>>>>>>>>>>> better
>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of builder
>>>> or
>>>>>>>>> build
>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>> are a
>>>>>>>>>> little
>>>>>>>>>>>>> less
>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>> necessarily
>>>>>>>>>> knows
>>>>>>>>>>>>> what
>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>>>>>>> (`table()`
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>> (`KTable`
>>>>>>>>> in
>>>>>>>>>>> this
>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above, but
>>>> it
>>>>>>>>>> doesn't
>>>>>>>>>>>>> read
>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>> damian.guy@gmail.com
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>> choices
>>>>>>>>>>> we've
>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>> operations
>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>> API,
>>>>>>>>> i.e,
>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>> becoming
>>>>>>>>> noisy
>>>>>>>>>>> and
>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>>>>>>> params.
>>>>>>>>>> In
>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>> caching
>>>>>>>>> off,
>>>>>>>>>> or
>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note this can
>>>> be
>>>>>>>>> done
>>>>>>>>>>> now
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we change the
>>>> DSL
>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>> easily
>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>> providing
>>>>>>>>> the
>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>> method,
>>>>>>>>>> so
>>>>>>>>>>> it
>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>>>> you
>>>>> on
>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>> but
>>>>> the
>>>>>>>>>> same
>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>> many
>>>>> join
>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>
>>>>>>>>>> -- 
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>
>>>
>>>
>>> -- 
>>> -- Guozhang
>>>
>>
>>
> 
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
It's too issues we want to tackle

 - too many overload (for some method we have already more than 10(
 - improve custom store API

-Matthias


On 7/7/17 3:42 PM, Jan Filipiak wrote:
> It makes me want to cry.
> 
> why on earth is the DSL going to expose all its implementation details now?
> especially being materialized or not.
> 
> If we want to take usefull steps in that direction maybe we are looking
> for a way to let the user switch back and forth between PAPI and DSL?
> 
> A change as the proposed would not eliminate any of my pain points while
> still being a heck of work migrating towards to.
> 
> Since I am only following this from the point where Eno CC'ed it into
> the users list:
> 
> Can someone please rephrase for me what problem this is trying to solve?
> I don't mean to be rude but It uses a problematic feature
> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
> us nowhere in making the configs more flexible, its just syntactic sugar.
> 
> A low effort shoot like: lets add a properties to operations that would
> otherwise become overloaded to heavy? Or pull the configs by some naming
> schema
> form the overall properties. Additionally to that we get rid of
> StateStoreSuppliers in the DSL and have them also configured by said
> properties.
> 
> => way easier to migrate to, way less risk, way more flexible in the
> future (different implementations of the same operation don't require
> code change to configure)
> 
> Line 184 makes especially no sense to me. what is a KTableKTable non
> materialized join anyways?
> 
> Hope we can discuss more on this.
> 
> 
> 
> On 07.07.2017 17:23, Guozhang Wang wrote:
>> I messed the indentation on github code repos; this would be easier to
>> read:
>>
>> https://codeshare.io/GLWW8K
>>
>>
>> Guozhang
>>
>>
>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Hi Damian / Kyle,
>>>
>>> I think I agree with you guys about the pros / cons of using the builder
>>> pattern v.s. using some "secondary classes". And I'm thinking if we can
>>> take a "mid" manner between these two. I spent some time with a slight
>>> different approach from Damian's current proposal:
>>>
>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>
>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>
>>> The key idea is to tolerate the final "table()" or "stream()"
>>> function to
>>> "upgrade" from the secondary classes to the first citizen classes, while
>>> having all the specs inside this function. Also this proposal
>>> includes some
>>> other refactoring that people have been discussed about for the
>>> builder to
>>> reduce the overloaded functions as well. WDYT?
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>>>
>>>> Hi Jan,
>>>>
>>>> Thanks very much for the input.
>>>>
>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>>> Hi Damian,
>>>>>
>>>>> I do see your point of something needs to change. But I fully agree
>>>>> with
>>>>> Gouzhang when he says.
>>>>> ---
>>>>>
>>>>> But since this is a incompatibility change, and we are going to remove
>>>> the
>>>>> compatibility annotations soon it means we only have one chance and we
>>>>> really have to make it right.
>>>>> ----
>>>>>
>>>>>
>>>> I think we all agree on this one! Hence the discussion.
>>>>
>>>>
>>>>> I fear all suggestions do not go far enough to become something that
>>>> will
>>>>> carry on for very much longer.
>>>>> I am currently working on KAFKA-3705 and try to find the most easy way
>>>> for
>>>>> the user to give me all the required functionality. The easiest
>>>> interface I
>>>>> could come up so far can be looked at here.
>>>>>
>>>>>
>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>>
>>>> And its already horribly complicated. I am currently unable to find the
>>>>> right abstraction level to have everything falling into place
>>>> naturally. To
>>>>> be honest I already think introducing
>>>>>
>>>>>
>>>> To be fair that is not a particularly easy problem to solve!
>>>>
>>>>
>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>> was unideal and makes everything a mess.
>>>>
>>>> I'm not sure i agree that it makes everything a mess, but It could have
>>>> been done differently.
>>>>
>>>> The JoinType:Whatever is also not really flexible. 2 things come to my
>>>> mind:
>>>>> 1. I don't think we should rule out config based decisions say configs
>>>> like
>>>>>          streams.$applicationID.joins.$joinname.conf = value
>>>>>
>>>> Is this just for config? Or are you suggesting that we could somehow
>>>> "code"
>>>> the join in a config file?
>>>>
>>>>
>>>>> This can allow for tremendous changes without single API change and
>>>>> IMO
>>>> it
>>>>> was not considered enough yet.
>>>>>
>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>>>>> example can be used to implement different join types as the user
>>>> wishes.
>>>> Do you have an example of how this might look?
>>>>
>>>>
>>>>> As Gouzhang said: stopping to break users is very important.
>>>>
>>>> Of course. We want to make it as easy as possible for people to use
>>>> streams.
>>>>
>>>>
>>>> especially with this changes + All the plans I sadly only have in my
>>>> head
>>>>> but hopefully the first link can give a glimpse.
>>>>>
>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>> exactly
>>>>> we are talking about. I would argue to go a bit slower and more
>>>> carefull on
>>>>> this one. At some point we need to get it right. Peeking over to the
>>>> hadoop
>>>>> guys with their hughe userbase. Config files really work well for
>>>>> them.
>>>>>
>>>>> Best Jan
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>> Thanks Matthias
>>>>>>
>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>>>>> wrote:
>>>>>>> I am just catching up on this thread, so sorry for the long email in
>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>> always a
>>>>>>> clear proposal. Still need to think about this in more detail. But
>>>> maybe
>>>>>>> it helps other to get new ideas :)
>>>>>>>
>>>>>>>
>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>> aggregate()
>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>> parameters
>>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>> might
>>>>>>>>> be quite confusion for developers.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> I see what you are saying, but the grouped stream is effectively a
>>>>> no-op
>>>>>>>> until you call one of the aggregate/count/reduce etc functions. So
>>>> the
>>>>>>>> optional params are ones that are applicable to any of the
>>>> operations
>>>>> you
>>>>>>>> can perform on this grouped stream. Then the final
>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>> required/specific to that function.
>>>>>>>>
>>>>>>> I understand your argument, but you don't share the conclusion.
>>>>>>> If we
>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>
>>>>>>> .groupBy().count().withXX().build()
>>>>>>>
>>>>>>> (with a better name for build() though)
>>>>>>>
>>>>>>>
>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>> apply
>>>>>> too all the aggregate functions. The terminal call being the actual
>>>> type
>>>>> of
>>>>>> aggregation you want to do. I personally find this more natural than
>>>>>> groupBy().count().withBlah().build()
>>>>>>
>>>>>>
>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>> sessionWindowed(). We should have one top-level `Windows` interface
>>>> that
>>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>> single
>>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>> separation of `SessionWindows` in the first place, and this seems to
>>>> be
>>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>> introduced session windows)
>>>>>>>
>>>>>> Yes - true we should look into that.
>>>>>>
>>>>>>
>>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and
>>>>>>> thus we
>>>>>>> might also want to use windowBy() (instead of windowed()). Not sure
>>>> how
>>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>
>>>>>>>
>>>>>> Makes sense
>>>>>>
>>>>>>
>>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>> think,
>>>>>>> defining an inner/left/outer join is not an optional argument but a
>>>>>>> first class concept and should have a proper representation in the
>>>> API
>>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>
>>>>>>>
>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>> what
>>>>> we
>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>
>>>>>>
>>>>>>
>>>>>>> About the two join API proposals, the second one has too much boiler
>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>> one
>>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>> operator call, should have one parameter per mandatory argument but
>>>> your
>>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>> call.
>>>>>>> This is far from intuitive IMHO.
>>>>>>>
>>>>>>>
>>>>>> This is the builder pattern, you only need one param as the builder
>>>> has
>>>>>> captured all of the required and optional arguments.
>>>>>>
>>>>>>
>>>>>>> The first join proposal also seems to align better with the pattern
>>>>>>> suggested for aggregations and having the same pattern for all
>>>> operators
>>>>>>> is important (as you stated already).
>>>>>>>
>>>>>>>
>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>> the
>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>
>>>>>>
>>>>>>> Coming back to the config vs optional parameter. What about having a
>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>
>>>>>>>
>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>> thing
>>>>> that
>>>>>> is really config.
>>>>>>
>>>>>>
>>>>>>> This also raises the question if until() is a windows property?
>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>> should
>>>>>>> not not have it's own method.
>>>>>>>
>>>>>>>
>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>> to be
>>>>>> potentially different for every window operation you do in a streams
>>>> app.
>>>>>>
>>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>
>>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>    groupedStream.counting()
>>>>>>>>                    .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>                    .table();
>>>>>>> This is an interesting idea, and it remind my on some feedback about
>>>> "I
>>>>>>> wanted to count a stream, but there was no count() method -- I first
>>>>>>> needed to figure out, that I need to group the stream first to be
>>>> able
>>>>>>> to count it. It does make sense in hindsight but was not obvious in
>>>> the
>>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>> following:
>>>>>>>
>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>
>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative here,
>>>> as
>>>>>>> it comes after the count()
>>>>>>>
>>>>>>> This would be more consistent than your proposal (that has grouping
>>>>>>> before but windowing after count()). It might even allow us to
>>>>>>> enrich
>>>>>>> the API with a some syntactic sugar like `stream.count().table()` to
>>>> get
>>>>>>> the overall count of all records (this would obviously not scale,
>>>> but we
>>>>>>> could support it -- if not now, maybe later).
>>>>>>>
>>>>>>>
>>>>>> I guess i'd prefer
>>>>>> stream.groupBy().windowBy().count()
>>>>>> stream.groupBy().windowBy().reduce()
>>>>>> stream.groupBy().count()
>>>>>>
>>>>>> As i said above, everything that happens before the final aggregate
>>>> call
>>>>>> can be applied to any of them. So it makes sense to me to do those
>>>> things
>>>>>> ahead of the final aggregate call.
>>>>>>
>>>>>>
>>>>>>> Last about builder pattern. I am convinced that we need some
>>>> "terminal"
>>>>>>> operator/method that tells us when to add the processor to the
>>>> topology.
>>>>>>> But I don't see the need for a plain builder pattern that feels
>>>> alien to
>>>>>>> me (see my argument about the second join proposal). Using .stream()
>>>> /
>>>>>>> .table() as use in many examples might work. But maybe a more
>>>>>>> generic
>>>>>>> name that we can use in all places like build() or apply() might
>>>> also be
>>>>>>> an option.
>>>>>>>
>>>>>>>
>>>>>> Sure, a generic name might be ok.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>> Thanks Kyle.
>>>>>>>>
>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>> winkelman.kyle@gmail.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi Damian,
>>>>>>>>>
>>>>>>>>>>>>> When trying to program in the fluent API that has been
>>>> discussed
>>>>>>> most
>>>>>>>>> it
>>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>>> you
>>>>> can
>>>>>>>>> reuse.
>>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>> legal
>>>>>>> to
>>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>> time?
>>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>> re-use it
>>>>>>> as
>>>>>>>>> you
>>>>>>>>>>> can today.
>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I
>>>> see
>>>>>>> it
>>>>>>>>> you wouldn’t be able to reuse anything except KStreams and
>>>>>>>>> KTables,
>>>>>>> because
>>>>>>>>> most of this fluent api would continue returning this (this being
>>>> the
>>>>>>>>> builder object currently being manipulated).
>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>> KTables
>>>>>>> and
>>>>>>>>> you use it in two different ways then its possible you make
>>>>> conflicting
>>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> No necessarily true. It could return a new instance of the builder,
>>>>> i.e.,
>>>>>>>> the builders being immutable. So if you held a reference to the
>>>> builder
>>>>>>> it
>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>
>>>>>>>>
>>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>> kStream.grouped();
>>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>
>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going to do
>>>> it
>>>>>>>>> eventually…
>>>>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>>>>> would
>>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>> work
>>>>>>>>> around would be to always make copies of the config objects you
>>>>>>>>> are
>>>>>>>>> building, but this approach has its own problem because now we
>>>> have to
>>>>>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>>>>> processors.
>>>>>>>>>
>>>>>>>>> The point of this long winded example is that we always have to be
>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>> user
>>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>
>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>
>>>>>>>>
>>>>>>>>> In my attempt at a couple methods with builders I feel that I
>>>>>>>>> could
>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>
>>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>>>>> Count
>>>>>>>>> for some reason it would throw an error message saying that a
>>>>>>>>> store
>>>>>>> named
>>>>>>>>> “my-store” already exists.
>>>>>>>>>
>>>>>>>>>
>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Kyle
>>>>>>>>>
>>>>>>>>> From: Damian Guy
>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>
>>>>>>>>> Hi Kyle,
>>>>>>>>>
>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>
>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>> winkelman.kyle@gmail.com
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>> against
>>>>>>>>>> it. The reason I like it is because it makes it clear to the user
>>>>> that
>>>>>>> a
>>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>> intermediate
>>>>>>>>>> class that I need to undetstand.
>>>>>>>>>>
>>>>>>>>> Yes, that makes sense.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> When trying to program in the fluent API that has been discussed
>>>> most
>>>>>>> it
>>>>>>>>>> feels difficult to know when you will actually get an object you
>>>> can
>>>>>>>>> reuse.
>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>> legal
>>>>> to
>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>> time?
>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
>>>> it
>>>>> as
>>>>>>> you
>>>>>>>>> can today.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>> pattern
>>>>> I
>>>>>>>>>> assume I can reuse everything.
>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a
>>>>>>>>>> big
>>>>> fan
>>>>>>> of
>>>>>>>>>> the grouped.
>>>>>>>>>>
>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>> and
>>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>> want to
>>>>>>> be
>>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this
>>>>>>>>> all
>>>>>>> depends
>>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>> (probably
>>>>> 3)
>>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>>> KTable
>>>>>>> with
>>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>> sessionwindowed
>>>>>>>>> for
>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>> Count,
>>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>> builder:
>>>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>>> // Windowed Count
>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>
>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>>> // Session Count
>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>
>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> // Reduce
>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>
>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>>> KTable<String, String> cogrouped =
>>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>>           .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>           .aggregate(initializer, aggregator,
>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> public class Count {
>>>>>>>>>>
>>>>>>>>>>       public static class Windowed extends Count {
>>>>>>>>>>           private Windows windows;
>>>>>>>>>>       }
>>>>>>>>>>       public static class SessionWindowed extends Count {
>>>>>>>>>>           private SessionWindows sessionWindows;
>>>>>>>>>>       }
>>>>>>>>>>
>>>>>>>>>>       public static Count count();
>>>>>>>>>>       public static Windowed windowed(Windows windows);
>>>>>>>>>>       public static SessionWindowed
>>>>>>>>>> sessionWindowed(SessionWindows
>>>>>>>>>> sessionWindows);
>>>>>>>>>>
>>>>>>>>>>       // All withXXX(...) methods.
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>       public KTable<K, Long> count(Count count);
>>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>> count);
>>>>>>>>>>       public KTable<Windowed<K>, Long>
>>>>>>>>>> count(Count.SessionWindowed
>>>>>>> count);
>>>>>>>>>> …
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Kyle
>>>>>>>>>>
>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>
>>>>>>>>>> I played the current proposal a bit with
>>>>>>> https://github.com/dguy/kafka/
>>>>>>>>>> tree/dsl-experiment <
>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>> ,
>>>>>>>>>> and here are my observations:
>>>>>>>>>>
>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>
>>>>>>>>>>       "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>
>>>>>>>>>> than
>>>>>>>>>>
>>>>>>>>>>       "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>
>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it
>>>> is
>>>>>>> not
>>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>> natural.
>>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>
>>>>>>>>>>
>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>>> )
>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>>>>> pass
>>>>>>> in
>>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>> return
>>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer
>>>>>>>>>> supplier
>>>>> with
>>>>>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>>>>> require
>>>>>>>>>> users pass in the inner state store supplier while specifying
>>>>> caching /
>>>>>>>>>> logging through other APIs.
>>>>>>>>>>
>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>>>> me:
>>>>> we
>>>>>>>>> are
>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>> multiple
>>>>>>> time,
>>>>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why
>>>> is
>>>>>>> that?
>>>>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>>>>> joins?
>>>>>>>>>>
>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>> topology
>>>>>>> so
>>>>>>>>> I
>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>>>> the
>>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Below are minor ones:
>>>>>>>>>>
>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>
>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>> matthias@confluent.io>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> I see your point about "when to add the processor to the
>>>> topology".
>>>>>>>>> That
>>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>>>>> topology...
>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>> interface
>>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> However, I don't understand your argument about putting
>>>> aggregate()
>>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>> parameters
>>>>>>>>>>> for aggregate() and not for groupBy() -- but a
>>>>>>>>>>> groupBy().withXX()
>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>> might
>>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>>> I also think that mixing optional parameters with configs is a
>>>> bad
>>>>>>>>>> idea.
>>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention it.
>>>> Hope
>>>>>>>>> to
>>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>> config
>>>>>>>>>> here
>>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>> already
>>>>>>>>> be
>>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>> .groupBy(...)
>>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could
>>>>>>>>>>>>> rename to
>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we
>>>>>>>>>>>>> should
>>>>> find
>>>>>>>>>>>>> some better names).
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>> close
>>>>>>>>>>>>> together. I would rather separate both more than less, ie, do
>>>> into
>>>>>>>>> the
>>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>> The
>>>>>>>>>> initial
>>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>
>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>> KTable
>>>>>>>>> which
>>>>>>>>>>> is
>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>> "build"
>>>>>>>>>> the
>>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>> optional
>>>>>>>>>>>>> parameter for count() have to specified on the .grouped() call
>>>> -- 
>>>>>>>>> this
>>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>> stream
>>>>>>>>> that
>>>>>>>>>>> you
>>>>>>>>>>>> will aggregate. So
>>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>>> etc
>>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>>> the
>>>>>>>>>> initial
>>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>>
>>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>>
>>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>>> .withXXX()
>>>>>>>>>> are
>>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything why
>>>> this
>>>>>>>>>> wont'
>>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>> See above.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>
>>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>>>>>>> fluent
>>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>>>> in
>>>>>>>>> the
>>>>>>>>>>> DSL,
>>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>>> things
>>>>>>>>>> that
>>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>>> withLogConfig(...)
>>>>>>>>> (even
>>>>>>>>>>>>>> though it is below) as this is actually config and we already
>>>>> have
>>>>>>>>> a
>>>>>>>>>>> way
>>>>>>>>>>>>> of
>>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>>>> the
>>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it
>>>>>>>>>>>>>> stands
>>>>> that
>>>>>>>>>> is
>>>>>>>>>>> a
>>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>>> the
>>>>>>>>>> default
>>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>>> to....
>>>>> If
>>>>>>>>>> you
>>>>>>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>>>>>>> logged
>>>>>>>>>>> etc,
>>>>>>>>>>>>>> you would currently need to implement that yourself. Ideally
>>>> we
>>>>> can
>>>>>>>>>>>>> provide
>>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>>> metrics,
>>>>>>>>>> etc. I
>>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>>> appropriate
>>>>>>>>>> on
>>>>>>>>>>>>> the
>>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>>           .withKeyMapper(keyMapper)
>>>>>>>>>>>>>>           .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>>           .withValueSerde(Serdes.String())
>>>>>>>>>>>>>>           .withQueryableName("my-store")
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>>           .reduce(windowedReducer);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>>           .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>>> = null;
>>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>>> stream.grouped()
>>>>>>>>>>>>>>           .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>>           .aggregate(aggregator, init);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("name")
>>>>>>>>>>>>>>           .withCachingEnabled(false)
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withQueryableName("q")
>>>>>>>>>>>>>>           .withLoggingEnabled(false)
>>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>>           .withLogConfig(Collections.sin
>>>> gletonMap("segment.size",
>>>>>>>>>> "10"))
>>>>>>>>>>>>>>           .reduce(reducer);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> If anyone wants to play around with this you can find the
>>>>>>>>>>>>>> code
>>>>>>>>> here:
>>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>>> return
>>>>>>>>> null.
>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>>>> both
>>>>>>>>> are
>>>>>>>>>>>>> better
>>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The fluent API approach reads better, no mention of builder
>>>> or
>>>>>>>>> build
>>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>>> are a
>>>>>>>>>> little
>>>>>>>>>>>>> less
>>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>>> necessarily
>>>>>>>>>> knows
>>>>>>>>>>>>> what
>>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>>>>>>> (`table()`
>>>>>>>>>>>>> in
>>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>>> (`KTable`
>>>>>>>>> in
>>>>>>>>>>> this
>>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above, but
>>>> it
>>>>>>>>>> doesn't
>>>>>>>>>>>>> read
>>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>>> damian.guy@gmail.com
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>>> choices
>>>>>>>>>>> we've
>>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to
>>>>>>>>>>>>>>>> stateful
>>>>>>>>>>> operations
>>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>>> API,
>>>>>>>>> i.e,
>>>>>>>>>>>>> there
>>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is
>>>>>>>>>>>>>>>> becoming
>>>>>>>>> noisy
>>>>>>>>>>> and
>>>>>>>>>>>>> i
>>>>>>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>>>>>>> params.
>>>>>>>>>> In
>>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>>> caching
>>>>>>>>> off,
>>>>>>>>>> or
>>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note this can
>>>> be
>>>>>>>>> done
>>>>>>>>>>> now
>>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>>> So this is a bit of an open question. How can we change the
>>>> DSL
>>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>>> easily
>>>>>>>>>>>>> extended
>>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>>> providing
>>>>>>>>> the
>>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>>> method,
>>>>>>>>>> so
>>>>>>>>>>> it
>>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>>>> you
>>>>> on
>>>>>>>>>>>>> about!
>>>>>>>>>>>>>>>> The above has focussed on state store related overloads,
>>>>>>>>>>>>>>>> but
>>>>> the
>>>>>>>>>> same
>>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have
>>>>>>>>>>>>>>>> many
>>>>> join
>>>>>>>>>>>>> methods
>>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>
>>>>>>>>>> -- 
>>>>>>>>>> -- Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>
>>>
>>>
>>> -- 
>>> -- Guozhang
>>>
>>
>>
> 
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Damian,

thanks for taking the time. I think you read my points individually but 
you seem to not understand the bigger picture I am trying to paint.

 From the three problems I mentioned - and that you agreed on to be 
problems -  you are only trying to address the first.

What I am trying to tell you is that if you focus on the later two the 
first one comes for free. On the other hand if you focus on the first
and please allow me to call it the easy part. All you going to archive 
is to break user land and sugar coat the real problems.

This takes away overloads, but still leaves it a mess to implement new 
features. I am currently trying to prep a patch for Kafka-3705 and
I do not understand why I should deal with Interactive Queries what so 
ever. My Output table has a proper ValueGetterSupplier.
That should be it!

I hope I made clear that to improve here quite some hard work has been 
done and that it would be rewariding and that just sugar coating everything
is one of the worst steps we could take from where we are at the moment.

Looking at Kafka-5581 that you mentioned. None of the points are really 
related to what I am saying really. Each of these points is tricky and
requires some carefull thinking but might work out.

Further Looking at you comment that refers to KIP vs. DISCUSS. I don't 
know what I should understand from that.

Regarding your comment mentioning that getQueryHandle() wouldn't work. 
Its the same thing as giving the user a queryable string.
It works the same way with the only difference that we have a wrapper 
object that gives the user what he wants instantly! Instead of giving 
him a String
to get a Store, we just give him a store, plus we don't hand out some 
inflexible native types that we later on don't have control over.
The whole logic about partitioners and what else does not change.

Hope this makes my points more clear.

Best Jan


On 19.07.2017 12:03, Damian Guy wrote:
> Hi Jan,
>
> Thanks for your input. Comments inline
>
> On Tue, 18 Jul 2017 at 15:21 Jan Filipiak <Ja...@trivago.com> wrote:
>
>> Hi,
>>
>>
>> 1. To many overloads:
>> Currently, whenever a KTable is the result of an operation it gets and
>> override with stateStoreName, and StatestoreSupplier in case people want
>> to query that.
>> This is what produces 2/3rd of the overloaded methods right now (not
>> counting methods returning KStream)
>>
>>
> As you state further down we are trying to address this.
>
>
>> 2. Code copy and pasting.
>> Almost all KTableProcessorSuppliers have the same block of (if(name !=
>> null) store.put(k,v))
>>
>>
> Yes, i agree. That is related to the KTable queryable store etc, and can
> easily be addressed, but isn't necessarily part of this as it doesn't need
> to be a public interface change, i.e., we can clean that up in the
> background.
>
>
>> 3. Runtime inefficiencies.
>> Each querable table almost instantly causes a another store beeing
>> required. Storing equivalent data of upstream KTables.
>>
> Agreed. Again, this is not a public interface change. We don't need another
> store, i.e., we can just use a "View" on the existing store, which is
> basically just using the KTableValueGetter to apply the map or filter
> operation to the original store. We also have this jira
> https://issues.apache.org/jira/browse/KAFKA-5581 to look into optimizing
> when we do and don't need to add additional changelogs.
>
>
>> So I really see us tackeling only the first part currently. Wich in my
>> opinion is to short-sighted to settle on an Public API.
>>
> We are not settling on the public API. We do, however need to do KIPs for
> public API discussions. For internal changes we don't necessarily need to
> have a public discussion about it.
>
>
>> This is why I want to tackle our approach to IQ-first, as it seems to me
>> to be the most disruptive thing. And the cause of most problems.
>>
>> The Plan:
>>
>> Table from topic, kstream (don't even like this one, but probaly needed
>> for some kind of enhanced flexibility) or aggregations would be the only
>> KTables that would get associated with a statestore (their processors).
>> For these operations one can have the "statestoresupplier" overload but
>> also not the "querablestatestore" overload. From this point on KTables
>> abstraction would be considered restored.
>> All the overloads of join and through with respect to IQ would go away.
>> "through" would go completely maybe the benefit added is. The method I
>> would add is for a table to get a Queryhandle.
>> This query handle will underneath remember its tables processor name. To
>> access the data form IQ we would not rely on the "per processor
>> statestore" but go the usual path through ValueGetterSupplier.
>> *Note:* We do not necessarily have a Serde for V, especially after
>> mapValues. also not for any intermediate Data types. It would be each
>> KTableProccesors job to provide a serialized version of upstream Datatypes.
>> KTableKTabkeJoinwould need to bring a JoinInputSerializer<V1,V2> that
>> would serialize both upstream values for transport across boxes.
>>
>> This first step would kill all the "Storename" based overloads + many
>> Statestore overloads. It would also avoid the bloated copy pasting in
>> each KTableProcessor for maintaining the store.
>> It would also make the runtime more efficient in a way that it does not
>> store the same data twice, just for accessing from IQ. Tackeling problem
>> 1 but also all other three problems mentioned above.
>>
>>   From here ~3 or 4 (from kstream,topic or aggregate) methods would still
>> be stuck with StateStoresupplier overload. For me, this is quite an
>> improvement already, to reduce further overloads I am thinking
>> to put a nullable properties to this operations. If people want to use
>> all defaults they could throw in null and it wouldn't be to painfull.
>> That doesn't necessarily require
>> them to have config files laying around. They could if they wanted use
>> property files to create such properties + we would over to look for
>> configs in the streams property.
>> So the complexity of distributing property files is optional and the
>> user might choose to fill the configs by code or files.
>>
>> I think these steps can rescue the proper abstraction of a KTable. I
>> believe that with the current proposals we are only sugarcoating problem
>> 1 and end up with a broken idea of what KTable is.
>> I think it will be even harder to develop further from there. Interface
>> wise my proposal is like developing backwards as i am very certain we
>> did a wrong turn with the IQ we shouldn't try to carry through.
>>
>> I hope I could explain how this re factoring can tackle  the 3 above
>> problems and especially why i don't think we can win tackiling only
>> point 1 in the long run.
>> If anything would need an implementation draft please feel free to ask
>> me to provide one. Initially the proposal hopefully would get the job
>> done of just removing clutter.
>>
>>
> I agree with some of what you have said in the above few paragraphs. I
> think you are correct in that KTable has become littered with a bunch of
> methods to make each stage queryable, i.e, adding the overloads
> for queryableStoreName and StateStoreSupplier. I think we can do away with
> both of them as once you have a KTable you can always build a view of it by
> using the KTableValueGetter. So we don't ever need a StateStoreSupplier as
> we have one already from when the original KTable was created. We can also
> possibly remove the overloads with queryableName and always use a generated
> name that can be retrieved from the method `String queryableStoreName` -
> this can then be used with IQ if needed.
>
> The getQueryHandle idea you mention won't really work as things stand. The
> KTable knows nothing about it's runtime context it is purely for building a
> topology that can be executed. In order to successfully query a `KTable`
> (state store) we need to know how many partitions and on which threads the
> state stores are running. This is why we added the `stores` API to
> `KafkaStreams` as this is the execution environment that has all of the
> information.
>
>
> Thanks,
> Damian
>
> Looking forward to your comments.
>> Best Jan
>>
>>
>>
>> On 12.07.2017 21:27, Guozhang Wang wrote:
>>> Hello Jan,
>>>
>>> Thanks for your feedbacks. Let me try to clarify a few things with the
>>> problems that we are trying to resolve and the motivations with the
>>> current proposals.
>>>
>>> As Matthias mentioned, one issue that we are trying to tackle is to
>>> reduce the number of overloaded functions in the DSL due to serde
>>> overridden / state store supplier overridden that are needed for
>>> repartition, or for state store materializations. Another related
>>> issue is that the current overridden state store supplier is not very
>>> natural to use, for example:
>>>
>>> 1) If a user just want to disable caching / logging etc but do not
>>> want to change the underlying store engine at all, she needs to learn
>>> to know that, for example, if a windowed store or key-value store is
>>> needed for this specific operator in the DSL, what serdes are needed
>>> for materialize the store, in order to create a StateStoreSupplier
>>> with caching / logging disabled, and then pass into the DSL.
>>>
>>> 2) Similarly, if a user just want to set different topic configs for
>>> the changelog topic, she still need to specify the whole
>>> StateStoreSupplier into the operator.
>>>
>>> 3) If a user want to use a different store engine (e.g. MyStore than
>>> RocksDBStore) underneath but do not care about the default settings
>>> for logging, caching, etc, he STILL needs to pass in the whole
>>> StateStoreSupplier into the operator.
>>>
>>> Note that all the above scenarios are for advanced users who do want
>>> to override these settings, for users who are just OK with the default
>>> settings they should be not exposed with such APIs at all, like you
>>> said, "I do not be exposed with any of such implementation details",
>>> if you do not care.
>>>
>>> -----------------
>>>
>>> We have been talking about the configs v.s. code for such settings,
>>> since we have been using configs for "global" default configs; but the
>>> arguments against using configs for such per-operator / per-store
>>> settings as well is that it will simply make configs hard to manage /
>>> hard to wire with tools. Personally speaking, I'm not a big fan of
>>> using configs for per-entity overrides and that is mainly from my
>>> experience with Samza:Samza inherits exactly the same approach for
>>> per-stream / per-source configs:
>>>
>>>
>> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html
>>> <
>> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html>
>> ([system-name][stream-id]
>>> etc are all place-holders)
>>>
>>> The main issues were 1) users making config changes need to deploy
>>> this to all the instances, I think for Streams it would be even worse
>>> as we need to make a config file on each of the running instance, and
>>> whenever there is a change we need to make sure they are propagated to
>>> all of them, 2) whenever users make some code changes, e.g. to add a
>>> new stream / system, they need to remember to set the corresponding
>>> changes in the config files as well and they kept forgetting about it,
>>> the lesson learned there was that it is always better to change one
>>> place (code change) than two (code change + config file change).
>>>
>>> Again, this is not saying we have vetoed this option, and if people
>>> have good reasons for this let's discuss them here.
>>>
>>> -----------------
>>>
>>> So the current proposals are mainly around keeping configs for the
>>> global default settings, while still allowing users to override
>>> per-operator / per-store settings in the code, while also keeping in
>>> mind to not forced users to think about such implementation details if
>>> they are fine with whatever the default settings. For example:
>>>
>>> As a normal user it is sufficient to specify an aggregation as
>>>
>>> ```
>>> table4.join(table5, joiner).table();
>>> ```
>>>
>>> in which she can still just focus on the computational logic with all
>>> implementation details abstracted away; only if the user are capable
>>> enough with the implementation details (e.g. how is the joining tables
>>> be materialized into state stores, etc) and want to specify her own
>>> settings (e.g. I want to swap in my own state store engine, or I want
>>> to disable caching for dedup, or use a different serde etc) she can
>>> "explore" them with the DSL again:
>>>
>>> ```
>>> table4.join(table5, joiner).table(Materialized.as("store1")); // use a
>>> custom store name for interactive query
>>> table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier));
>>> // use a custom store engine
>>> table4.join(table5,
>>> joiner).table(Materialized.as("store1").withLoggingEnabled(configs));
>>> // use a custom store changelog topic configs
>>> // ... more
>>> ```
>>>
>>> Hope it helps.
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>>> <ma...@trivago.com>> wrote:
>>>
>>>      It makes me want to cry.
>>>
>>>      why on earth is the DSL going to expose all its implementation
>>>      details now?
>>>      especially being materialized or not.
>>>
>>>      If we want to take usefull steps in that direction maybe we are
>>>      looking for a way to let the user switch back and forth between
>>>      PAPI and DSL?
>>>
>>>      A change as the proposed would not eliminate any of my pain points
>>>      while still being a heck of work migrating towards to.
>>>
>>>      Since I am only following this from the point where Eno CC'ed it
>>>      into the users list:
>>>
>>>      Can someone please rephrase for me what problem this is trying to
>>>      solve? I don't mean to be rude but It uses a problematic feature
>>>      "StateStoreSuppliers in DSL" to justify making it even worse. This
>>>      helps us nowhere in making the configs more flexible, its just
>>>      syntactic sugar.
>>>
>>>      A low effort shoot like: lets add a properties to operations that
>>>      would otherwise become overloaded to heavy? Or pull the configs by
>>>      some naming schema
>>>      form the overall properties. Additionally to that we get rid of
>>>      StateStoreSuppliers in the DSL and have them also configured by
>>>      said properties.
>>>
>>>      => way easier to migrate to, way less risk, way more flexible in
>>>      the future (different implementations of the same operation don't
>>>      require code change to configure)
>>>
>>>      Line 184 makes especially no sense to me. what is a KTableKTable
>>>      non materialized join anyways?
>>>
>>>      Hope we can discuss more on this.
>>>
>>>
>>>
>>>
>>>      On 07.07.2017 17:23, Guozhang Wang wrote:
>>>
>>>          I messed the indentation on github code repos; this would be
>>>          easier to read:
>>>
>>>          https://codeshare.io/GLWW8K
>>>
>>>
>>>          Guozhang
>>>
>>>
>>>          On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang
>>>          <wangguoz@gmail.com <ma...@gmail.com>> wrote:
>>>
>>>              Hi Damian / Kyle,
>>>
>>>              I think I agree with you guys about the pros / cons of
>>>              using the builder
>>>              pattern v.s. using some "secondary classes". And I'm
>>>              thinking if we can
>>>              take a "mid" manner between these two. I spent some time
>>>              with a slight
>>>              different approach from Damian's current proposal:
>>>
>>>
>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>>>              <
>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/>
>>>              java/org/apache/kafka/streams/RefactoredAPIs.java
>>>
>>>              The key idea is to tolerate the final "table()" or
>>>              "stream()" function to
>>>              "upgrade" from the secondary classes to the first citizen
>>>              classes, while
>>>              having all the specs inside this function. Also this
>>>              proposal includes some
>>>              other refactoring that people have been discussed about
>>>              for the builder to
>>>              reduce the overloaded functions as well. WDYT?
>>>
>>>
>>>              Guozhang
>>>
>>>
>>>              On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy
>>>              <damian.guy@gmail.com <ma...@gmail.com>> wrote:
>>>
>>>                  Hi Jan,
>>>
>>>                  Thanks very much for the input.
>>>
>>>                  On Tue, 4 Jul 2017 at 08:54 Jan Filipiak
>>>                  <Jan.Filipiak@trivago.com
>>>                  <ma...@trivago.com>>
>>>                  wrote:
>>>
>>>                      Hi Damian,
>>>
>>>                      I do see your point of something needs to change.
>>>                      But I fully agree with
>>>                      Gouzhang when he says.
>>>                      ---
>>>
>>>                      But since this is a incompatibility change, and we
>>>                      are going to remove
>>>
>>>                  the
>>>
>>>                      compatibility annotations soon it means we only
>>>                      have one chance and we
>>>                      really have to make it right.
>>>                      ----
>>>
>>>
>>>                  I think we all agree on this one! Hence the discussion.
>>>
>>>
>>>                      I fear all suggestions do not go far enough to
>>>                      become something that
>>>
>>>                  will
>>>
>>>                      carry on for very much longer.
>>>                      I am currently working on KAFKA-3705 and try to
>>>                      find the most easy way
>>>
>>>                  for
>>>
>>>                      the user to give me all the required
>>>                      functionality. The easiest
>>>
>>>                  interface I
>>>
>>>                      could come up so far can be looked at here.
>>>
>>>
>>>
>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>                      <
>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>>>                  de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>                  kafka/streams/kstream/internals/KTableImpl.java#L622
>>>
>>>
>>>                  And its already horribly complicated. I am currently
>>>                  unable to find the
>>>
>>>                      right abstraction level to have everything falling
>>>                      into place
>>>
>>>                  naturally. To
>>>
>>>                      be honest I already think introducing
>>>
>>>
>>>                  To be fair that is not a particularly easy problem to
>>>                  solve!
>>>
>>>
>>>
>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>                      <
>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>>>                  de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>                  kafka/streams/kstream/internals/KTableImpl.java#L493
>>>
>>>                      was unideal and makes everything a mess.
>>>
>>>
>>>                  I'm not sure i agree that it makes everything a mess,
>>>                  but It could have
>>>                  been done differently.
>>>
>>>                  The JoinType:Whatever is also not really flexible. 2
>>>                  things come to my
>>>                  mind:
>>>
>>>                      1. I don't think we should rule out config based
>>>                      decisions say configs
>>>
>>>                  like
>>>
>>>
>>>                       streams.$applicationID.joins.$joinname.conf = value
>>>
>>>                  Is this just for config? Or are you suggesting that we
>>>                  could somehow
>>>                  "code"
>>>                  the join in a config file?
>>>
>>>
>>>                      This can allow for tremendous changes without
>>>                      single API change and IMO
>>>
>>>                  it
>>>
>>>                      was not considered enough yet.
>>>
>>>                      2. Push logic from the DSL to the Callback
>>>                      classes. A ValueJoiner for
>>>                      example can be used to implement different join
>>>                      types as the user
>>>
>>>                  wishes.
>>>                  Do you have an example of how this might look?
>>>
>>>
>>>                      As Gouzhang said: stopping to break users is very
>>>                      important.
>>>
>>>
>>>                  Of course. We want to make it as easy as possible for
>>>                  people to use
>>>                  streams.
>>>
>>>
>>>                  especially with this changes + All the plans I sadly
>>>                  only have in my head
>>>
>>>                      but hopefully the first link can give a glimpse.
>>>
>>>                      Thanks for preparing the examples made it way
>>>                      clearer to me what exactly
>>>                      we are talking about. I would argue to go a bit
>>>                      slower and more
>>>
>>>                  carefull on
>>>
>>>                      this one. At some point we need to get it right.
>>>                      Peeking over to the
>>>
>>>                  hadoop
>>>
>>>                      guys with their hughe userbase. Config files
>>>                      really work well for them.
>>>
>>>                      Best Jan
>>>
>>>
>>>
>>>
>>>
>>>                      On 30.06.2017 09:31, Damian Guy wrote:
>>>
>>>                          Thanks Matthias
>>>
>>>                          On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>>>                          <matthias@confluent.io
>>>                          <ma...@confluent.io>>
>>>
>>>                      wrote:
>>>
>>>                              I am just catching up on this thread, so
>>>                              sorry for the long email in
>>>                              advance... Also, it's to some extend a
>>>                              dump of thoughts and not
>>>
>>>                  always a
>>>
>>>                              clear proposal. Still need to think about
>>>                              this in more detail. But
>>>
>>>                  maybe
>>>
>>>                              it helps other to get new ideas :)
>>>
>>>
>>>                                      However, I don't understand your
>>>                                      argument about putting aggregate()
>>>                                      after the withXX() -- all the
>>>                                      calls to withXX() set optional
>>>
>>>                      parameters
>>>
>>>                                      for aggregate() and not for
>>>                                      groupBy() -- but a groupBy().withXX()
>>>                                      indicates that the withXX()
>>>                                      belongs to the groupBy(). IMHO, this
>>>
>>>                  might
>>>
>>>                                      be quite confusion for developers.
>>>
>>>
>>>                                  I see what you are saying, but the
>>>                                  grouped stream is effectively a
>>>
>>>                      no-op
>>>
>>>                                  until you call one of the
>>>                                  aggregate/count/reduce etc functions. So
>>>
>>>                  the
>>>
>>>                                  optional params are ones that are
>>>                                  applicable to any of the
>>>
>>>                  operations
>>>
>>>                      you
>>>
>>>                                  can perform on this grouped stream.
>>>                                  Then the final
>>>                                  count()/reduce()/aggregate() call has
>>>                                  any of the params that are
>>>                                  required/specific to that function.
>>>
>>>                              I understand your argument, but you don't
>>>                              share the conclusion. If we
>>>                              need a "final/terminal" call, the better
>>>                              way might be
>>>
>>>                              .groupBy().count().withXX().build()
>>>
>>>                              (with a better name for build() though)
>>>
>>>
>>>                          The point is that all the other calls,
>>>                          i.e,withBlah, windowed, etc
>>>
>>>                  apply
>>>
>>>                          too all the aggregate functions. The terminal
>>>                          call being the actual
>>>
>>>                  type
>>>
>>>                      of
>>>
>>>                          aggregation you want to do. I personally find
>>>                          this more natural than
>>>                          groupBy().count().withBlah().build()
>>>
>>>
>>>                                  groupedStream.count(/** non windowed
>>>                                  count**/)
>>>
>>   groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>   groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>                              I like this. However, I don't see a reason
>>>                              to have windowed() and
>>>                              sessionWindowed(). We should have one
>>>                              top-level `Windows` interface
>>>
>>>                  that
>>>
>>>                              both `TimeWindows` and `SessionWindows`
>>>                              implement and just have a
>>>
>>>                  single
>>>
>>>                              windowed() method that accepts all
>>>                              `Windows`. (I did not like the
>>>                              separation of `SessionWindows` in the
>>>                              first place, and this seems to
>>>
>>>                  be
>>>
>>>                              an opportunity to clean this up. It was
>>>                              hard to change when we
>>>                              introduced session windows)
>>>
>>>                          Yes - true we should look into that.
>>>
>>>
>>>                              Btw: we do you the imperative groupBy()
>>>                              and groupByKey(), and thus we
>>>                              might also want to use windowBy() (instead
>>>                              of windowed()). Not sure
>>>
>>>                  how
>>>
>>>                              important this is, but it seems to be
>>>                              inconsistent otherwise.
>>>
>>>
>>>                          Makes sense
>>>
>>>
>>>                              About joins:  I don't like
>>>                              .withJoinType(JoinType.LEFT) at all. I
>>>
>>>                  think,
>>>
>>>                              defining an inner/left/outer join is not
>>>                              an optional argument but a
>>>                              first class concept and should have a
>>>                              proper representation in the
>>>
>>>                  API
>>>
>>>                              (like the current methods join(),
>>>                              leftJoin, outerJoin()).
>>>
>>>
>>>                          Yep, i did originally have it as a required
>>>                          param and maybe that is
>>>
>>>                  what
>>>
>>>                      we
>>>
>>>                          go with. It could have a default, but maybe
>>>                          that is confusing.
>>>
>>>
>>>
>>>                              About the two join API proposals, the
>>>                              second one has too much boiler
>>>                              plate code for my taste. Also, the actual
>>>                              join() operator has only
>>>
>>>                  one
>>>
>>>                              argument what is weird to me, as in my
>>>                              thinking process, the main
>>>                              operator call, should have one parameter
>>>                              per mandatory argument but
>>>
>>>                  your
>>>
>>>                              proposal put the mandatory arguments into
>>>                              Joins.streamStreamJoin()
>>>
>>>                  call.
>>>
>>>                              This is far from intuitive IMHO.
>>>
>>>
>>>                          This is the builder pattern, you only need one
>>>                          param as the builder
>>>
>>>                  has
>>>
>>>                          captured all of the required and optional
>>>                          arguments.
>>>
>>>
>>>                              The first join proposal also seems to
>>>                              align better with the pattern
>>>                              suggested for aggregations and having the
>>>                              same pattern for all
>>>
>>>                  operators
>>>
>>>                              is important (as you stated already).
>>>
>>>
>>>                          This is why i offered two alternatives as i
>>>                          started out with. 1 is the
>>>                          builder pattern, the other is the more fluent
>>>                          pattern.
>>>
>>>
>>>                              Coming back to the config vs optional
>>>                              parameter. What about having a
>>>                              method withConfig[s](...) that allow to
>>>                              put in the configuration?
>>>
>>>
>>>                          Sure, it is currently called withLogConfig()
>>>                          as that is the only thing
>>>
>>>                      that
>>>
>>>                          is really config.
>>>
>>>
>>>                              This also raises the question if until()
>>>                              is a windows property?
>>>                              Actually, until() seems to be a
>>>                              configuration parameter and thus,
>>>
>>>                  should
>>>
>>>                              not not have it's own method.
>>>
>>>
>>>                          Hmmm, i don't agree. Until is a property of
>>>                          the window. It is going
>>>
>>>                  to be
>>>
>>>                          potentially different for every window
>>>                          operation you do in a streams
>>>
>>>                  app.
>>>
>>>
>>>                              Browsing throw your example DSL branch, I
>>>                              also saw this one:
>>>
>>>                                  final KTable<Windowed<String>, Long>
>>>                                  windowed>
>>>
>>>                                 groupedStream.counting()
>>>
>>>                                   .windowed(TimeWindows.of(10L).until(10))
>>>                                                     .table();
>>>
>>>                              This is an interesting idea, and it remind
>>>                              my on some feedback about
>>>
>>>                  "I
>>>
>>>                              wanted to count a stream, but there was no
>>>                              count() method -- I first
>>>                              needed to figure out, that I need to group
>>>                              the stream first to be
>>>
>>>                  able
>>>
>>>                              to count it. It does make sense in
>>>                              hindsight but was not obvious in
>>>
>>>                  the
>>>
>>>                              beginning". Thus, carrying out this
>>>                              thought, we could also do the
>>>                              following:
>>>
>>>
>>   stream.count().groupedBy().windowedBy().table();
>>>                              -> Note, I use "grouped" and "windowed"
>>>                              instead of imperative here,
>>>
>>>                  as
>>>
>>>                              it comes after the count()
>>>
>>>                              This would be more consistent than your
>>>                              proposal (that has grouping
>>>                              before but windowing after count()). It
>>>                              might even allow us to enrich
>>>                              the API with a some syntactic sugar like
>>>                              `stream.count().table()` to
>>>
>>>                  get
>>>
>>>                              the overall count of all records (this
>>>                              would obviously not scale,
>>>
>>>                  but we
>>>
>>>                              could support it -- if not now, maybe later).
>>>
>>>
>>>                          I guess i'd prefer
>>>                          stream.groupBy().windowBy().count()
>>>                          stream.groupBy().windowBy().reduce()
>>>                          stream.groupBy().count()
>>>
>>>                          As i said above, everything that happens
>>>                          before the final aggregate
>>>
>>>                  call
>>>
>>>                          can be applied to any of them. So it makes
>>>                          sense to me to do those
>>>
>>>                  things
>>>
>>>                          ahead of the final aggregate call.
>>>
>>>
>>>                              Last about builder pattern. I am convinced
>>>                              that we need some
>>>
>>>                  "terminal"
>>>
>>>                              operator/method that tells us when to add
>>>                              the processor to the
>>>
>>>                  topology.
>>>
>>>                              But I don't see the need for a plain
>>>                              builder pattern that feels
>>>
>>>                  alien to
>>>
>>>                              me (see my argument about the second join
>>>                              proposal). Using .stream()
>>>
>>>                  /
>>>
>>>                              .table() as use in many examples might
>>>                              work. But maybe a more generic
>>>                              name that we can use in all places like
>>>                              build() or apply() might
>>>
>>>                  also be
>>>
>>>                              an option.
>>>
>>>
>>>                          Sure, a generic name might be ok.
>>>
>>>
>>>
>>>
>>>                              -Matthias
>>>
>>>
>>>
>>>                              On 6/29/17 7:37 AM, Damian Guy wrote:
>>>
>>>                                  Thanks Kyle.
>>>
>>>                                  On Thu, 29 Jun 2017 at 15:11 Kyle
>>>                                  Winkelman <
>>>
>>>                  winkelman.kyle@gmail.com
>>>                  <ma...@gmail.com>>
>>>
>>>                                  wrote:
>>>
>>>                                      Hi Damian,
>>>
>>>                                                      When trying to
>>>                                                      program in the
>>>                                                      fluent API that
>>>                                                      has been
>>>
>>>                  discussed
>>>
>>>                              most
>>>
>>>                                      it
>>>
>>>                                                      feels difficult to
>>>                                                      know when you will
>>>                                                      actually get an
>> object
>>>                  you
>>>
>>>                      can
>>>
>>>                                      reuse.
>>>
>>>                                                      What if I make one
>>>                                                      KGroupedStream
>>>                                                      that I want to
>>>                                                      reuse, is it
>>>
>>>                      legal
>>>
>>>                              to
>>>
>>>                                                      reuse it or does
>>>                                                      this approach
>>>                                                      expect you to call
>>>                                                      grouped each
>>>
>>>                      time?
>>>
>>>                                              I'd anticipate that once
>>>                                              you have a KGroupedStream
>>>                                              you can
>>>
>>>                  re-use it
>>>
>>>                              as
>>>
>>>                                      you
>>>
>>>                                              can today.
>>>
>>>                                      You said it yourself in another
>>>                                      post that the grouped stream is
>>>                                      effectively a no-op until a count,
>>>                                      reduce, or aggregate. The way I
>>>
>>>                  see
>>>
>>>                              it
>>>
>>>                                      you wouldn’t be able to reuse
>>>                                      anything except KStreams and KTables,
>>>
>>>                              because
>>>
>>>                                      most of this fluent api would
>>>                                      continue returning this (this being
>>>
>>>                  the
>>>
>>>                                      builder object currently being
>>>                                      manipulated).
>>>
>>>                                  So, if you ever store a reference to
>>>                                  anything but KStreams and
>>>
>>>                  KTables
>>>
>>>                              and
>>>
>>>                                      you use it in two different ways
>>>                                      then its possible you make
>>>
>>>                      conflicting
>>>
>>>                                      withXXX() calls on the same builder.
>>>
>>>
>>>                                  No necessarily true. It could return a
>>>                                  new instance of the builder,
>>>
>>>                      i.e.,
>>>
>>>                                  the builders being immutable. So if
>>>                                  you held a reference to the
>>>
>>>                  builder
>>>
>>>                              it
>>>
>>>                                  would always be the same as it was
>>>                                  when it was created.
>>>
>>>
>>>                                      GroupedStream<K,V>
>>>                                      groupedStreamWithDefaultSerdes =
>>>
>>>                  kStream.grouped();
>>>
>>>                                      GroupedStream<K,V>
>>>                                      groupedStreamWithDeclaredSerdes =
>>>
>>   groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>                                      I’ll admit that this shouldn’t
>>>                                      happen but some user is going to do
>>>
>>>                  it
>>>
>>>                                      eventually…
>>>                                      Depending on implementation uses
>>>                                      of groupedStreamWithDefaultSerdes
>>>
>>>                      would
>>>
>>>                                      most likely be equivalent to the
>>>                                      version withDeclaredSerdes. One
>>>
>>>                  work
>>>
>>>                                      around would be to always make
>>>                                      copies of the config objects you are
>>>                                      building, but this approach has
>>>                                      its own problem because now we
>>>
>>>                  have to
>>>
>>>                                      identify which configs are
>>>                                      equivalent so we don’t create
>> repeated
>>>                                      processors.
>>>
>>>                                      The point of this long winded
>>>                                      example is that we always have to be
>>>                                      thinking about all of the possible
>>>                                      ways it could be misused by a
>>>
>>>                  user
>>>
>>>                                      (causing them to see hard to
>>>                                      diagnose problems).
>>>
>>>                                  Exactly! That is the point of the
>>>                                  discussion really.
>>>
>>>
>>>                                      In my attempt at a couple methods
>>>                                      with builders I feel that I could
>>>                                      confidently say the user couldn’t
>>>                                      really mess it up.
>>>
>>>                                          // Count
>>>                                          KTable<String, Long> count =
>>>
>>>
>>   kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>                                      The kGroupedStream is reusable and
>>>                                      if they attempted to reuse the
>>>
>>>                      Count
>>>
>>>                                      for some reason it would throw an
>>>                                      error message saying that a store
>>>
>>>                              named
>>>
>>>                                      “my-store” already exists.
>>>
>>>
>>>                                  Yes i agree and i think using builders
>>>                                  is my preferred pattern.
>>>
>>>                                  Cheers,
>>>                                  Damian
>>>
>>>
>>>                                      Thanks,
>>>                                      Kyle
>>>
>>>                                      From: Damian Guy
>>>                                      Sent: Thursday, June 29, 2017 3:59 AM
>>>                                      To: dev@kafka.apache.org
>>>                                      <ma...@kafka.apache.org>
>>>                                      Subject: Re: [DISCUSS] Streams
>>>                                      DSL/StateStore Refactoring
>>>
>>>                                      Hi Kyle,
>>>
>>>                                      Thanks for your input. Really
>>>                                      appreciated.
>>>
>>>                                      On Thu, 29 Jun 2017 at 06:09 Kyle
>>>                                      Winkelman <
>>>
>>>                  winkelman.kyle@gmail.com <mailto:
>> winkelman.kyle@gmail.com>
>>>                                      wrote:
>>>
>>>                                          I like more of a builder
>>>                                          pattern even though others
>>>                                          have voiced
>>>
>>>                      against
>>>
>>>                                          it. The reason I like it is
>>>                                          because it makes it clear to
>>>                                          the user
>>>
>>>                      that
>>>
>>>                              a
>>>
>>>                                          call to KGroupedStream#count
>>>                                          will return a KTable not some
>>>
>>>                      intermediate
>>>
>>>                                          class that I need to undetstand.
>>>
>>>                                      Yes, that makes sense.
>>>
>>>
>>>                                          When trying to program in the
>>>                                          fluent API that has been
>> discussed
>>>                  most
>>>
>>>                              it
>>>
>>>                                          feels difficult to know when
>>>                                          you will actually get an
>>>                                          object you
>>>
>>>                  can
>>>
>>>                                      reuse.
>>>
>>>                                          What if I make one
>>>                                          KGroupedStream that I want to
>>>                                          reuse, is it
>>>
>>>                  legal
>>>
>>>                      to
>>>
>>>                                          reuse it or does this approach
>>>                                          expect you to call grouped each
>>>
>>>                  time?
>>>
>>>                                      I'd anticipate that once you have
>>>                                      a KGroupedStream you can re-use
>>>
>>>                  it
>>>
>>>                      as
>>>
>>>                              you
>>>
>>>                                      can today.
>>>
>>>
>>>                                          This question doesn’t pop into
>>>                                          my head at all in the builder
>>>
>>>                  pattern
>>>
>>>                      I
>>>
>>>                                          assume I can reuse everything.
>>>                                          Finally, I like .groupByKey
>>>                                          and .groupBy(KeyValueMapper)
>>>                                          not a big
>>>
>>>                      fan
>>>
>>>                              of
>>>
>>>                                          the grouped.
>>>
>>>                                          Yes, grouped() was more for
>>>                                          demonstration and because
>>>                                          groupBy()
>>>
>>>                  and
>>>
>>>                                      groupByKey() were taken! So i'd
>>>                                      imagine the api would actually
>>>
>>>                  want to
>>>
>>>                              be
>>>
>>>                                      groupByKey(/** no required
>>>                                      args***/).withOptionalArg() and
>>>                                      groupBy(KeyValueMapper
>>>                                      m).withOpitionalArg(...)  of
>>>                                      course this all
>>>
>>>                              depends
>>>
>>>                                      on maintaining backward
>> compatibility.
>>>
>>>                                          Unfortunately, the below
>>>                                          approach would require atleast 2
>>>
>>>                  (probably
>>>
>>>                      3)
>>>
>>>                                          overloads (one for returning a
>>>                                          KTable and one for returning a
>>>
>>>                  KTable
>>>
>>>                              with
>>>
>>>                                          Windowed Key, probably would
>>>                                          want to split windowed and
>>>
>>>                      sessionwindowed
>>>
>>>                                      for
>>>
>>>                                          ease of implementation) of
>>>                                          each count, reduce, and
>> aggregate.
>>>                                          Obviously not exhaustive but
>>>                                          enough for you to get the
>> picture.
>>>                      Count,
>>>
>>>                                          Reduce, and Aggregate supply 3
>>>                                          static methods to initialize the
>>>
>>>                              builder:
>>>
>>>                                          // Count
>>>                                          KTable<String, Long> count =
>>>
>>>
>>   groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>                                          // Windowed Count
>>>                                          KTable<Windowed<String>, Long>
>>>                                          windowedCount =
>>>
>>>
>>   groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>                  (10)).withQueryableStoreName("my-windowed-store"));
>>>
>>>                                          // Session Count
>>>                                          KTable<Windowed<String>, Long>
>>>                                          sessionCount =
>>>
>>>
>>   groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>
>>   with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>                                      Above and below, i think i'd
>>>                                      prefer it to be:
>>>                                      groupedStream.count(/** non
>>>                                      windowed count**/)
>>>
>>   groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>   groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>
>>>
>>>
>>>                                          // Reduce
>>>                                          Reducer<Long> reducer;
>>>                                          KTable<String, Long> reduce =
>>>                                          groupedStream.reduce(reducer,
>>>
>>   Reduce.reduce().withQueryableStoreName("my-store"));
>>>                                          // Aggregate Windowed with
>>>                                          Custom Store
>>>                                          Initializer<String> initializer;
>>>                                          Aggregator<String, Long,
>>>                                          String> aggregator;
>>>                                          KTable<Windowed<String>,
>>>                                          String> aggregate =
>>>
>>   groupedStream.aggregate(initializer,
>>>                                          aggregator,
>>>
>>>                      Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>
>>>                  withStateStoreSupplier(stateStoreSupplier)));
>>>
>>>                                          // Cogroup SessionWindowed
>>>                                          KTable<String, String> cogrouped
>> =
>>>                      groupedStream1.cogroup(aggregator1)
>>>
>>>
>>>                                          .cogroup(groupedStream2,
>>>                                          aggregator2)
>>>
>>>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Just to add to Damian's points, I think the optimizations for KAFKA-5581
can be done as a second step after this API refactoring, since it should
not require any further public API changes and hence to me a bit orthogonal
to this KIP. To illustrate my thinking:

```
someOps.table(); // the library may or may not materialize the resulted
KTable in a new store depending on the
// optimizations we have; but that is abstracted away from the user and
even if
// there is a materialized store, it is not exposed for querying.

someOps.table(Materialized.as("store1")) // the library may or may not
ACTUALLY materialize the resulted KTable
// in a new store depending on the optimizations;
// if it ever decides to materialize it will choose whatever the default
// storage engine, but it will expose either the actual state store or
// just the "logical view" for users to query.

someOps.table(Materialized.as(MyStoreSupplier)) // the library was forced
by user to materialize the resulted KTable
// with the provided store supplier's storage engine; the users can
// always query it with the supplier's name reference.
```

So, the only case that we have to materialize the store is when user did
enforce a store supplier indicating that she wants to physically
materialize it with the provided storage engine; in other cases the library
still have the freedom to choose whether to materialize or not.


Guozhang


On Wed, Jul 19, 2017 at 3:03 AM, Damian Guy <da...@gmail.com> wrote:

> Hi Jan,
>
> Thanks for your input. Comments inline
>
> On Tue, 18 Jul 2017 at 15:21 Jan Filipiak <Ja...@trivago.com>
> wrote:
>
>> Hi,
>>
>>
>> 1. To many overloads:
>> Currently, whenever a KTable is the result of an operation it gets and
>> override with stateStoreName, and StatestoreSupplier in case people want
>> to query that.
>> This is what produces 2/3rd of the overloaded methods right now (not
>> counting methods returning KStream)
>>
>>
> As you state further down we are trying to address this.
>
>
>> 2. Code copy and pasting.
>> Almost all KTableProcessorSuppliers have the same block of (if(name !=
>> null) store.put(k,v))
>>
>>
> Yes, i agree. That is related to the KTable queryable store etc, and can
> easily be addressed, but isn't necessarily part of this as it doesn't need
> to be a public interface change, i.e., we can clean that up in the
> background.
>
>
>> 3. Runtime inefficiencies.
>> Each querable table almost instantly causes a another store beeing
>> required. Storing equivalent data of upstream KTables.
>>
>
> Agreed. Again, this is not a public interface change. We don't need
> another store, i.e., we can just use a "View" on the existing store, which
> is basically just using the KTableValueGetter to apply the map or filter
> operation to the original store. We also have this jira
> https://issues.apache.org/jira/browse/KAFKA-5581 to look into optimizing
> when we do and don't need to add additional changelogs.
>
>
>>
>> So I really see us tackeling only the first part currently. Wich in my
>> opinion is to short-sighted to settle on an Public API.
>>
>
> We are not settling on the public API. We do, however need to do KIPs for
> public API discussions. For internal changes we don't necessarily need to
> have a public discussion about it.
>
>
>> This is why I want to tackle our approach to IQ-first, as it seems to me
>> to be the most disruptive thing. And the cause of most problems.
>>
>> The Plan:
>>
>> Table from topic, kstream (don't even like this one, but probaly needed
>> for some kind of enhanced flexibility) or aggregations would be the only
>> KTables that would get associated with a statestore (their processors).
>> For these operations one can have the "statestoresupplier" overload but
>> also not the "querablestatestore" overload. From this point on KTables
>> abstraction would be considered restored.
>> All the overloads of join and through with respect to IQ would go away.
>> "through" would go completely maybe the benefit added is. The method I
>> would add is for a table to get a Queryhandle.
>> This query handle will underneath remember its tables processor name. To
>> access the data form IQ we would not rely on the "per processor
>> statestore" but go the usual path through ValueGetterSupplier.
>> *Note:* We do not necessarily have a Serde for V, especially after
>>
>> mapValues. also not for any intermediate Data types. It would be each
>> KTableProccesors job to provide a serialized version of upstream
>> Datatypes.
>> KTableKTabkeJoinwould need to bring a JoinInputSerializer<V1,V2> that
>> would serialize both upstream values for transport across boxes.
>>
>> This first step would kill all the "Storename" based overloads + many
>> Statestore overloads. It would also avoid the bloated copy pasting in
>> each KTableProcessor for maintaining the store.
>> It would also make the runtime more efficient in a way that it does not
>> store the same data twice, just for accessing from IQ. Tackeling problem
>> 1 but also all other three problems mentioned above.
>>
>>  From here ~3 or 4 (from kstream,topic or aggregate) methods would still
>> be stuck with StateStoresupplier overload. For me, this is quite an
>> improvement already, to reduce further overloads I am thinking
>> to put a nullable properties to this operations. If people want to use
>> all defaults they could throw in null and it wouldn't be to painfull.
>> That doesn't necessarily require
>> them to have config files laying around. They could if they wanted use
>> property files to create such properties + we would over to look for
>> configs in the streams property.
>> So the complexity of distributing property files is optional and the
>> user might choose to fill the configs by code or files.
>>
>> I think these steps can rescue the proper abstraction of a KTable. I
>> believe that with the current proposals we are only sugarcoating problem
>> 1 and end up with a broken idea of what KTable is.
>> I think it will be even harder to develop further from there. Interface
>> wise my proposal is like developing backwards as i am very certain we
>> did a wrong turn with the IQ we shouldn't try to carry through.
>>
>> I hope I could explain how this re factoring can tackle  the 3 above
>> problems and especially why i don't think we can win tackiling only
>> point 1 in the long run.
>> If anything would need an implementation draft please feel free to ask
>> me to provide one. Initially the proposal hopefully would get the job
>> done of just removing clutter.
>>
>>
>
> I agree with some of what you have said in the above few paragraphs. I
> think you are correct in that KTable has become littered with a bunch of
> methods to make each stage queryable, i.e, adding the overloads
> for queryableStoreName and StateStoreSupplier. I think we can do away with
> both of them as once you have a KTable you can always build a view of it by
> using the KTableValueGetter. So we don't ever need a StateStoreSupplier as
> we have one already from when the original KTable was created. We can also
> possibly remove the overloads with queryableName and always use a generated
> name that can be retrieved from the method `String queryableStoreName` -
> this can then be used with IQ if needed.
>
> The getQueryHandle idea you mention won't really work as things stand. The
> KTable knows nothing about it's runtime context it is purely for building a
> topology that can be executed. In order to successfully query a `KTable`
> (state store) we need to know how many partitions and on which threads the
> state stores are running. This is why we added the `stores` API to
> `KafkaStreams` as this is the execution environment that has all of the
> information.
>
>
> Thanks,
> Damian
>
> Looking forward to your comments.
>>
>> Best Jan
>>
>>
>>
>> On 12.07.2017 21:27, Guozhang Wang wrote:
>> > Hello Jan,
>> >
>> > Thanks for your feedbacks. Let me try to clarify a few things with the
>> > problems that we are trying to resolve and the motivations with the
>> > current proposals.
>> >
>> > As Matthias mentioned, one issue that we are trying to tackle is to
>> > reduce the number of overloaded functions in the DSL due to serde
>> > overridden / state store supplier overridden that are needed for
>> > repartition, or for state store materializations. Another related
>> > issue is that the current overridden state store supplier is not very
>> > natural to use, for example:
>> >
>> > 1) If a user just want to disable caching / logging etc but do not
>> > want to change the underlying store engine at all, she needs to learn
>> > to know that, for example, if a windowed store or key-value store is
>> > needed for this specific operator in the DSL, what serdes are needed
>> > for materialize the store, in order to create a StateStoreSupplier
>> > with caching / logging disabled, and then pass into the DSL.
>> >
>> > 2) Similarly, if a user just want to set different topic configs for
>> > the changelog topic, she still need to specify the whole
>> > StateStoreSupplier into the operator.
>> >
>> > 3) If a user want to use a different store engine (e.g. MyStore than
>> > RocksDBStore) underneath but do not care about the default settings
>> > for logging, caching, etc, he STILL needs to pass in the whole
>> > StateStoreSupplier into the operator.
>> >
>> > Note that all the above scenarios are for advanced users who do want
>> > to override these settings, for users who are just OK with the default
>> > settings they should be not exposed with such APIs at all, like you
>> > said, "I do not be exposed with any of such implementation details",
>> > if you do not care.
>> >
>> > -----------------
>> >
>> > We have been talking about the configs v.s. code for such settings,
>> > since we have been using configs for "global" default configs; but the
>> > arguments against using configs for such per-operator / per-store
>> > settings as well is that it will simply make configs hard to manage /
>> > hard to wire with tools. Personally speaking, I'm not a big fan of
>> > using configs for per-entity overrides and that is mainly from my
>> > experience with Samza:Samza inherits exactly the same approach for
>> > per-stream / per-source configs:
>> >
>> > http://samza.apache.org/learn/documentation/0.13/jobs/
>> configuration-table.html
>> > <http://samza.apache.org/learn/documentation/0.13/jobs/
>> configuration-table.html> ([system-name][stream-id]
>> > etc are all place-holders)
>> >
>> > The main issues were 1) users making config changes need to deploy
>> > this to all the instances, I think for Streams it would be even worse
>> > as we need to make a config file on each of the running instance, and
>> > whenever there is a change we need to make sure they are propagated to
>> > all of them, 2) whenever users make some code changes, e.g. to add a
>> > new stream / system, they need to remember to set the corresponding
>> > changes in the config files as well and they kept forgetting about it,
>> > the lesson learned there was that it is always better to change one
>> > place (code change) than two (code change + config file change).
>> >
>> > Again, this is not saying we have vetoed this option, and if people
>> > have good reasons for this let's discuss them here.
>> >
>> > -----------------
>> >
>> > So the current proposals are mainly around keeping configs for the
>> > global default settings, while still allowing users to override
>> > per-operator / per-store settings in the code, while also keeping in
>> > mind to not forced users to think about such implementation details if
>> > they are fine with whatever the default settings. For example:
>> >
>> > As a normal user it is sufficient to specify an aggregation as
>> >
>> > ```
>> > table4.join(table5, joiner).table();
>> > ```
>> >
>> > in which she can still just focus on the computational logic with all
>> > implementation details abstracted away; only if the user are capable
>> > enough with the implementation details (e.g. how is the joining tables
>> > be materialized into state stores, etc) and want to specify her own
>> > settings (e.g. I want to swap in my own state store engine, or I want
>> > to disable caching for dedup, or use a different serde etc) she can
>> > "explore" them with the DSL again:
>> >
>> > ```
>> > table4.join(table5, joiner).table(Materialized.as("store1")); // use a
>> > custom store name for interactive query
>> > table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier));
>> > // use a custom store engine
>> > table4.join(table5,
>> > joiner).table(Materialized.as("store1").withLoggingEnabled(configs));
>> > // use a custom store changelog topic configs
>> > // ... more
>> > ```
>> >
>> > Hope it helps.
>> >
>> >
>> > Guozhang
>> >
>> >
>> > On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Jan.Filipiak@trivago.com
>> > <ma...@trivago.com>> wrote:
>> >
>> >     It makes me want to cry.
>> >
>> >     why on earth is the DSL going to expose all its implementation
>> >     details now?
>> >     especially being materialized or not.
>> >
>> >     If we want to take usefull steps in that direction maybe we are
>> >     looking for a way to let the user switch back and forth between
>> >     PAPI and DSL?
>> >
>> >     A change as the proposed would not eliminate any of my pain points
>> >     while still being a heck of work migrating towards to.
>> >
>> >     Since I am only following this from the point where Eno CC'ed it
>> >     into the users list:
>> >
>> >     Can someone please rephrase for me what problem this is trying to
>> >     solve? I don't mean to be rude but It uses a problematic feature
>> >     "StateStoreSuppliers in DSL" to justify making it even worse. This
>> >     helps us nowhere in making the configs more flexible, its just
>> >     syntactic sugar.
>> >
>> >     A low effort shoot like: lets add a properties to operations that
>> >     would otherwise become overloaded to heavy? Or pull the configs by
>> >     some naming schema
>> >     form the overall properties. Additionally to that we get rid of
>> >     StateStoreSuppliers in the DSL and have them also configured by
>> >     said properties.
>> >
>> >     => way easier to migrate to, way less risk, way more flexible in
>> >     the future (different implementations of the same operation don't
>> >     require code change to configure)
>> >
>> >     Line 184 makes especially no sense to me. what is a KTableKTable
>> >     non materialized join anyways?
>> >
>> >     Hope we can discuss more on this.
>> >
>> >
>> >
>> >
>> >     On 07.07.2017 17:23, Guozhang Wang wrote:
>> >
>> >         I messed the indentation on github code repos; this would be
>> >         easier to read:
>> >
>> >         https://codeshare.io/GLWW8K
>> >
>> >
>> >         Guozhang
>> >
>> >
>> >         On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang
>> >         <wangguoz@gmail.com <ma...@gmail.com>> wrote:
>> >
>> >             Hi Damian / Kyle,
>> >
>> >             I think I agree with you guys about the pros / cons of
>> >             using the builder
>> >             pattern v.s. using some "secondary classes". And I'm
>> >             thinking if we can
>> >             take a "mid" manner between these two. I spent some time
>> >             with a slight
>> >             different approach from Damian's current proposal:
>> >
>> >             https://github.com/guozhangwang/kafka/blob/dsl-
>> refactor/streams/src/main/
>> >             <https://github.com/guozhangwang/kafka/blob/dsl-
>> refactor/streams/src/main/>
>> >             java/org/apache/kafka/streams/RefactoredAPIs.java
>> >
>> >             The key idea is to tolerate the final "table()" or
>> >             "stream()" function to
>> >             "upgrade" from the secondary classes to the first citizen
>> >             classes, while
>> >             having all the specs inside this function. Also this
>> >             proposal includes some
>> >             other refactoring that people have been discussed about
>> >             for the builder to
>> >             reduce the overloaded functions as well. WDYT?
>> >
>> >
>> >             Guozhang
>> >
>> >
>> >             On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy
>> >             <damian.guy@gmail.com <ma...@gmail.com>> wrote:
>> >
>> >                 Hi Jan,
>> >
>> >                 Thanks very much for the input.
>> >
>> >                 On Tue, 4 Jul 2017 at 08:54 Jan Filipiak
>> >                 <Jan.Filipiak@trivago.com
>> >                 <ma...@trivago.com>>
>>
>> >                 wrote:
>> >
>> >                     Hi Damian,
>> >
>> >                     I do see your point of something needs to change.
>> >                     But I fully agree with
>> >                     Gouzhang when he says.
>> >                     ---
>> >
>> >                     But since this is a incompatibility change, and we
>> >                     are going to remove
>> >
>> >                 the
>> >
>> >                     compatibility annotations soon it means we only
>> >                     have one chance and we
>> >                     really have to make it right.
>> >                     ----
>> >
>> >
>> >                 I think we all agree on this one! Hence the discussion.
>> >
>> >
>> >                     I fear all suggestions do not go far enough to
>> >                     become something that
>> >
>> >                 will
>> >
>> >                     carry on for very much longer.
>> >                     I am currently working on KAFKA-3705 and try to
>> >                     find the most easy way
>> >
>> >                 for
>> >
>> >                     the user to give me all the required
>> >                     functionality. The easiest
>> >
>> >                 interface I
>> >
>> >                     could come up so far can be looked at here.
>> >
>> >
>> >                     https://github.com/Kaiserchen/kafka/blob/
>> 3da2b8f787a5d30dee2
>> >                     <https://github.com/Kaiserchen/kafka/blob/
>> 3da2b8f787a5d30dee2>
>> >
>> >                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> >                 kafka/streams/kstream/internals/KTableImpl.java#L622
>> >
>> >
>> >                 And its already horribly complicated. I am currently
>> >                 unable to find the
>> >
>> >                     right abstraction level to have everything falling
>> >                     into place
>> >
>> >                 naturally. To
>> >
>> >                     be honest I already think introducing
>> >
>> >
>> >                 To be fair that is not a particularly easy problem to
>> >                 solve!
>> >
>> >
>> >                     https://github.com/Kaiserchen/kafka/blob/
>> 3da2b8f787a5d30dee2
>> >                     <https://github.com/Kaiserchen/kafka/blob/
>> 3da2b8f787a5d30dee2>
>> >
>> >                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> >                 kafka/streams/kstream/internals/KTableImpl.java#L493
>> >
>> >                     was unideal and makes everything a mess.
>> >
>> >
>> >                 I'm not sure i agree that it makes everything a mess,
>> >                 but It could have
>> >                 been done differently.
>> >
>> >                 The JoinType:Whatever is also not really flexible. 2
>> >                 things come to my
>> >                 mind:
>> >
>> >                     1. I don't think we should rule out config based
>> >                     decisions say configs
>> >
>> >                 like
>> >
>> >
>> >                      streams.$applicationID.joins.$joinname.conf =
>> value
>> >
>> >                 Is this just for config? Or are you suggesting that we
>> >                 could somehow
>> >                 "code"
>> >                 the join in a config file?
>> >
>> >
>> >                     This can allow for tremendous changes without
>> >                     single API change and IMO
>> >
>> >                 it
>> >
>> >                     was not considered enough yet.
>> >
>> >                     2. Push logic from the DSL to the Callback
>> >                     classes. A ValueJoiner for
>> >                     example can be used to implement different join
>> >                     types as the user
>> >
>> >                 wishes.
>> >                 Do you have an example of how this might look?
>> >
>> >
>> >                     As Gouzhang said: stopping to break users is very
>> >                     important.
>> >
>> >
>> >                 Of course. We want to make it as easy as possible for
>> >                 people to use
>> >                 streams.
>> >
>> >
>> >                 especially with this changes + All the plans I sadly
>> >                 only have in my head
>> >
>> >                     but hopefully the first link can give a glimpse.
>> >
>> >                     Thanks for preparing the examples made it way
>> >                     clearer to me what exactly
>> >                     we are talking about. I would argue to go a bit
>> >                     slower and more
>> >
>> >                 carefull on
>> >
>> >                     this one. At some point we need to get it right.
>> >                     Peeking over to the
>> >
>> >                 hadoop
>> >
>> >                     guys with their hughe userbase. Config files
>> >                     really work well for them.
>> >
>> >                     Best Jan
>> >
>> >
>> >
>> >
>> >
>> >                     On 30.06.2017 09:31, Damian Guy wrote:
>> >
>> >                         Thanks Matthias
>> >
>> >                         On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>> >                         <matthias@confluent.io
>> >                         <ma...@confluent.io>>
>>
>> >
>> >                     wrote:
>> >
>> >                             I am just catching up on this thread, so
>> >                             sorry for the long email in
>> >                             advance... Also, it's to some extend a
>> >                             dump of thoughts and not
>> >
>> >                 always a
>> >
>> >                             clear proposal. Still need to think about
>> >                             this in more detail. But
>> >
>> >                 maybe
>> >
>> >                             it helps other to get new ideas :)
>> >
>> >
>> >                                     However, I don't understand your
>> >                                     argument about putting aggregate()
>> >                                     after the withXX() -- all the
>> >                                     calls to withXX() set optional
>> >
>> >                     parameters
>> >
>> >                                     for aggregate() and not for
>> >                                     groupBy() -- but a
>> groupBy().withXX()
>> >                                     indicates that the withXX()
>> >                                     belongs to the groupBy(). IMHO, this
>> >
>> >                 might
>> >
>> >                                     be quite confusion for developers.
>> >
>> >
>> >                                 I see what you are saying, but the
>> >                                 grouped stream is effectively a
>> >
>> >                     no-op
>> >
>> >                                 until you call one of the
>> >                                 aggregate/count/reduce etc functions. So
>> >
>> >                 the
>> >
>> >                                 optional params are ones that are
>> >                                 applicable to any of the
>> >
>> >                 operations
>> >
>> >                     you
>> >
>> >                                 can perform on this grouped stream.
>> >                                 Then the final
>> >                                 count()/reduce()/aggregate() call has
>> >                                 any of the params that are
>> >                                 required/specific to that function.
>> >
>> >                             I understand your argument, but you don't
>> >                             share the conclusion. If we
>> >                             need a "final/terminal" call, the better
>> >                             way might be
>> >
>> >                             .groupBy().count().withXX().build()
>> >
>> >                             (with a better name for build() though)
>> >
>> >
>> >                         The point is that all the other calls,
>> >                         i.e,withBlah, windowed, etc
>> >
>> >                 apply
>> >
>> >                         too all the aggregate functions. The terminal
>> >                         call being the actual
>> >
>> >                 type
>> >
>> >                     of
>> >
>> >                         aggregation you want to do. I personally find
>> >                         this more natural than
>> >                         groupBy().count().withBlah().build()
>> >
>> >
>> >                                 groupedStream.count(/** non windowed
>> >                                 count**/)
>> >                                 groupedStream.windowed(
>> TimeWindows.of(10L)).count(...)
>> >                                 groupedStream.sessionWindowed(
>> SessionWindows.of(10L)).count(...)
>> >
>> >                             I like this. However, I don't see a reason
>> >                             to have windowed() and
>> >                             sessionWindowed(). We should have one
>> >                             top-level `Windows` interface
>> >
>> >                 that
>> >
>> >                             both `TimeWindows` and `SessionWindows`
>> >                             implement and just have a
>> >
>> >                 single
>> >
>> >                             windowed() method that accepts all
>> >                             `Windows`. (I did not like the
>> >                             separation of `SessionWindows` in the
>> >                             first place, and this seems to
>> >
>> >                 be
>> >
>> >                             an opportunity to clean this up. It was
>> >                             hard to change when we
>> >                             introduced session windows)
>> >
>> >                         Yes - true we should look into that.
>> >
>> >
>> >                             Btw: we do you the imperative groupBy()
>> >                             and groupByKey(), and thus we
>> >                             might also want to use windowBy() (instead
>> >                             of windowed()). Not sure
>> >
>> >                 how
>> >
>> >                             important this is, but it seems to be
>> >                             inconsistent otherwise.
>> >
>> >
>> >                         Makes sense
>> >
>> >
>> >                             About joins:  I don't like
>> >                             .withJoinType(JoinType.LEFT) at all. I
>> >
>> >                 think,
>> >
>> >                             defining an inner/left/outer join is not
>> >                             an optional argument but a
>> >                             first class concept and should have a
>> >                             proper representation in the
>> >
>> >                 API
>> >
>> >                             (like the current methods join(),
>> >                             leftJoin, outerJoin()).
>> >
>> >
>> >                         Yep, i did originally have it as a required
>> >                         param and maybe that is
>> >
>> >                 what
>> >
>> >                     we
>> >
>> >                         go with. It could have a default, but maybe
>> >                         that is confusing.
>> >
>> >
>> >
>> >                             About the two join API proposals, the
>> >                             second one has too much boiler
>> >                             plate code for my taste. Also, the actual
>> >                             join() operator has only
>> >
>> >                 one
>> >
>> >                             argument what is weird to me, as in my
>> >                             thinking process, the main
>> >                             operator call, should have one parameter
>> >                             per mandatory argument but
>> >
>> >                 your
>> >
>> >                             proposal put the mandatory arguments into
>> >                             Joins.streamStreamJoin()
>> >
>> >                 call.
>> >
>> >                             This is far from intuitive IMHO.
>> >
>> >
>> >                         This is the builder pattern, you only need one
>> >                         param as the builder
>> >
>> >                 has
>> >
>> >                         captured all of the required and optional
>> >                         arguments.
>> >
>> >
>> >                             The first join proposal also seems to
>> >                             align better with the pattern
>> >                             suggested for aggregations and having the
>> >                             same pattern for all
>> >
>> >                 operators
>> >
>> >                             is important (as you stated already).
>> >
>> >
>> >                         This is why i offered two alternatives as i
>> >                         started out with. 1 is the
>> >                         builder pattern, the other is the more fluent
>> >                         pattern.
>> >
>> >
>> >                             Coming back to the config vs optional
>> >                             parameter. What about having a
>> >                             method withConfig[s](...) that allow to
>> >                             put in the configuration?
>> >
>> >
>> >                         Sure, it is currently called withLogConfig()
>> >                         as that is the only thing
>> >
>> >                     that
>> >
>> >                         is really config.
>> >
>> >
>> >                             This also raises the question if until()
>> >                             is a windows property?
>> >                             Actually, until() seems to be a
>> >                             configuration parameter and thus,
>> >
>> >                 should
>> >
>> >                             not not have it's own method.
>> >
>> >
>> >                         Hmmm, i don't agree. Until is a property of
>> >                         the window. It is going
>> >
>> >                 to be
>> >
>> >                         potentially different for every window
>> >                         operation you do in a streams
>> >
>> >                 app.
>> >
>> >
>> >                             Browsing throw your example DSL branch, I
>> >                             also saw this one:
>> >
>> >                                 final KTable<Windowed<String>, Long>
>> >                                 windowed>
>> >
>> >                                groupedStream.counting()
>> >
>> >                                  .windowed(TimeWindows.of(10L).
>> until(10))
>> >                                                    .table();
>> >
>> >                             This is an interesting idea, and it remind
>> >                             my on some feedback about
>> >
>> >                 "I
>> >
>> >                             wanted to count a stream, but there was no
>> >                             count() method -- I first
>> >                             needed to figure out, that I need to group
>> >                             the stream first to be
>> >
>> >                 able
>> >
>> >                             to count it. It does make sense in
>> >                             hindsight but was not obvious in
>> >
>> >                 the
>> >
>> >                             beginning". Thus, carrying out this
>> >                             thought, we could also do the
>> >                             following:
>> >
>> >                             stream.count().groupedBy().
>> windowedBy().table();
>> >
>> >                             -> Note, I use "grouped" and "windowed"
>> >                             instead of imperative here,
>> >
>> >                 as
>> >
>> >                             it comes after the count()
>> >
>> >                             This would be more consistent than your
>> >                             proposal (that has grouping
>> >                             before but windowing after count()). It
>> >                             might even allow us to enrich
>> >                             the API with a some syntactic sugar like
>> >                             `stream.count().table()` to
>> >
>> >                 get
>> >
>> >                             the overall count of all records (this
>> >                             would obviously not scale,
>> >
>> >                 but we
>> >
>> >                             could support it -- if not now, maybe
>> later).
>> >
>> >
>> >                         I guess i'd prefer
>> >                         stream.groupBy().windowBy().count()
>> >                         stream.groupBy().windowBy().reduce()
>> >                         stream.groupBy().count()
>> >
>> >                         As i said above, everything that happens
>> >                         before the final aggregate
>> >
>> >                 call
>> >
>> >                         can be applied to any of them. So it makes
>> >                         sense to me to do those
>> >
>> >                 things
>> >
>> >                         ahead of the final aggregate call.
>> >
>> >
>> >                             Last about builder pattern. I am convinced
>> >                             that we need some
>> >
>> >                 "terminal"
>> >
>> >                             operator/method that tells us when to add
>> >                             the processor to the
>> >
>> >                 topology.
>> >
>> >                             But I don't see the need for a plain
>> >                             builder pattern that feels
>> >
>> >                 alien to
>> >
>> >                             me (see my argument about the second join
>> >                             proposal). Using .stream()
>> >
>> >                 /
>> >
>> >                             .table() as use in many examples might
>> >                             work. But maybe a more generic
>> >                             name that we can use in all places like
>> >                             build() or apply() might
>> >
>> >                 also be
>> >
>> >                             an option.
>> >
>> >
>> >                         Sure, a generic name might be ok.
>> >
>> >
>> >
>> >
>> >                             -Matthias
>> >
>> >
>> >
>> >                             On 6/29/17 7:37 AM, Damian Guy wrote:
>> >
>> >                                 Thanks Kyle.
>> >
>> >                                 On Thu, 29 Jun 2017 at 15:11 Kyle
>> >                                 Winkelman <
>> >
>> >                 winkelman.kyle@gmail.com
>> >                 <ma...@gmail.com>>
>>
>> >
>> >                                 wrote:
>> >
>> >                                     Hi Damian,
>> >
>> >                                                     When trying to
>> >                                                     program in the
>> >                                                     fluent API that
>> >                                                     has been
>> >
>> >                 discussed
>> >
>> >                             most
>> >
>> >                                     it
>> >
>> >                                                     feels difficult to
>> >                                                     know when you will
>> >                                                     actually get an
>> object
>> >
>> >                 you
>> >
>> >                     can
>> >
>> >                                     reuse.
>> >
>> >                                                     What if I make one
>> >                                                     KGroupedStream
>> >                                                     that I want to
>> >                                                     reuse, is it
>> >
>> >                     legal
>> >
>> >                             to
>> >
>> >                                                     reuse it or does
>> >                                                     this approach
>> >                                                     expect you to call
>> >                                                     grouped each
>> >
>> >                     time?
>> >
>> >                                             I'd anticipate that once
>> >                                             you have a KGroupedStream
>> >                                             you can
>> >
>> >                 re-use it
>> >
>> >                             as
>> >
>> >                                     you
>> >
>> >                                             can today.
>> >
>> >                                     You said it yourself in another
>> >                                     post that the grouped stream is
>> >                                     effectively a no-op until a count,
>> >                                     reduce, or aggregate. The way I
>> >
>> >                 see
>> >
>> >                             it
>> >
>> >                                     you wouldn’t be able to reuse
>> >                                     anything except KStreams and
>> KTables,
>> >
>> >                             because
>> >
>> >                                     most of this fluent api would
>> >                                     continue returning this (this being
>> >
>> >                 the
>> >
>> >                                     builder object currently being
>> >                                     manipulated).
>> >
>> >                                 So, if you ever store a reference to
>> >                                 anything but KStreams and
>> >
>> >                 KTables
>> >
>> >                             and
>> >
>> >                                     you use it in two different ways
>> >                                     then its possible you make
>> >
>> >                     conflicting
>> >
>> >                                     withXXX() calls on the same builder.
>> >
>> >
>> >                                 No necessarily true. It could return a
>> >                                 new instance of the builder,
>> >
>> >                     i.e.,
>> >
>> >                                 the builders being immutable. So if
>> >                                 you held a reference to the
>> >
>> >                 builder
>> >
>> >                             it
>> >
>> >                                 would always be the same as it was
>> >                                 when it was created.
>> >
>> >
>> >                                     GroupedStream<K,V>
>> >                                     groupedStreamWithDefaultSerdes =
>> >
>> >                 kStream.grouped();
>> >
>> >                                     GroupedStream<K,V>
>> >                                     groupedStreamWithDeclaredSerdes =
>> >                                     groupedStreamsWithDefaultSerde
>> s.withKeySerde(…).withValueSerde(…);
>> >
>> >                                     I’ll admit that this shouldn’t
>> >                                     happen but some user is going to do
>> >
>> >                 it
>> >
>> >                                     eventually…
>> >                                     Depending on implementation uses
>> >                                     of groupedStreamWithDefaultSerdes
>> >
>> >                     would
>> >
>> >                                     most likely be equivalent to the
>> >                                     version withDeclaredSerdes. One
>> >
>> >                 work
>> >
>> >                                     around would be to always make
>> >                                     copies of the config objects you are
>> >                                     building, but this approach has
>> >                                     its own problem because now we
>> >
>> >                 have to
>> >
>> >                                     identify which configs are
>> >                                     equivalent so we don’t create
>> repeated
>> >                                     processors.
>> >
>> >                                     The point of this long winded
>> >                                     example is that we always have to be
>> >                                     thinking about all of the possible
>> >                                     ways it could be misused by a
>> >
>> >                 user
>> >
>> >                                     (causing them to see hard to
>> >                                     diagnose problems).
>> >
>> >                                 Exactly! That is the point of the
>> >                                 discussion really.
>> >
>> >
>> >                                     In my attempt at a couple methods
>> >                                     with builders I feel that I could
>> >                                     confidently say the user couldn’t
>> >                                     really mess it up.
>> >
>> >                                         // Count
>> >                                         KTable<String, Long> count =
>> >
>> >                     kGroupedStream.count(Count.count().
>> withQueryableStoreName("my-store"));
>> >
>> >                                     The kGroupedStream is reusable and
>> >                                     if they attempted to reuse the
>> >
>> >                     Count
>> >
>> >                                     for some reason it would throw an
>> >                                     error message saying that a store
>> >
>> >                             named
>> >
>> >                                     “my-store” already exists.
>> >
>> >
>> >                                 Yes i agree and i think using builders
>> >                                 is my preferred pattern.
>> >
>> >                                 Cheers,
>> >                                 Damian
>> >
>> >
>> >                                     Thanks,
>> >                                     Kyle
>> >
>> >                                     From: Damian Guy
>> >                                     Sent: Thursday, June 29, 2017 3:59
>> AM
>> >                                     To: dev@kafka.apache.org
>> >                                     <ma...@kafka.apache.org>
>> >                                     Subject: Re: [DISCUSS] Streams
>> >                                     DSL/StateStore Refactoring
>> >
>> >                                     Hi Kyle,
>> >
>> >                                     Thanks for your input. Really
>> >                                     appreciated.
>> >
>> >                                     On Thu, 29 Jun 2017 at 06:09 Kyle
>> >                                     Winkelman <
>> >
>> >                 winkelman.kyle@gmail.com <mailto:winkelman.kyle@gmail.
>> com>
>> >
>> >                                     wrote:
>> >
>> >                                         I like more of a builder
>> >                                         pattern even though others
>> >                                         have voiced
>> >
>> >                     against
>> >
>> >                                         it. The reason I like it is
>> >                                         because it makes it clear to
>> >                                         the user
>> >
>> >                     that
>> >
>> >                             a
>> >
>> >                                         call to KGroupedStream#count
>> >                                         will return a KTable not some
>> >
>> >                     intermediate
>> >
>> >                                         class that I need to undetstand.
>> >
>> >                                     Yes, that makes sense.
>> >
>> >
>> >                                         When trying to program in the
>> >                                         fluent API that has been
>> discussed
>> >
>> >                 most
>> >
>> >                             it
>> >
>> >                                         feels difficult to know when
>> >                                         you will actually get an
>> >                                         object you
>> >
>> >                 can
>> >
>> >                                     reuse.
>> >
>> >                                         What if I make one
>> >                                         KGroupedStream that I want to
>> >                                         reuse, is it
>> >
>> >                 legal
>> >
>> >                     to
>> >
>> >                                         reuse it or does this approach
>> >                                         expect you to call grouped each
>> >
>> >                 time?
>> >
>> >                                     I'd anticipate that once you have
>> >                                     a KGroupedStream you can re-use
>> >
>> >                 it
>> >
>> >                     as
>> >
>> >                             you
>> >
>> >                                     can today.
>> >
>> >
>> >                                         This question doesn’t pop into
>> >                                         my head at all in the builder
>> >
>> >                 pattern
>> >
>> >                     I
>> >
>> >                                         assume I can reuse everything.
>> >                                         Finally, I like .groupByKey
>> >                                         and .groupBy(KeyValueMapper)
>> >                                         not a big
>> >
>> >                     fan
>> >
>> >                             of
>> >
>> >                                         the grouped.
>> >
>> >                                         Yes, grouped() was more for
>> >                                         demonstration and because
>> >                                         groupBy()
>> >
>> >                 and
>> >
>> >                                     groupByKey() were taken! So i'd
>> >                                     imagine the api would actually
>> >
>> >                 want to
>> >
>> >                             be
>> >
>> >                                     groupByKey(/** no required
>> >                                     args***/).withOptionalArg() and
>> >                                     groupBy(KeyValueMapper
>> >                                     m).withOpitionalArg(...)  of
>> >                                     course this all
>> >
>> >                             depends
>> >
>> >                                     on maintaining backward
>> compatibility.
>> >
>> >
>> >                                         Unfortunately, the below
>> >                                         approach would require atleast 2
>> >
>> >                 (probably
>> >
>> >                     3)
>> >
>> >                                         overloads (one for returning a
>> >                                         KTable and one for returning a
>> >
>> >                 KTable
>> >
>> >                             with
>> >
>> >                                         Windowed Key, probably would
>> >                                         want to split windowed and
>> >
>> >                     sessionwindowed
>> >
>> >                                     for
>> >
>> >                                         ease of implementation) of
>> >                                         each count, reduce, and
>> aggregate.
>> >                                         Obviously not exhaustive but
>> >                                         enough for you to get the
>> picture.
>> >
>> >                     Count,
>> >
>> >                                         Reduce, and Aggregate supply 3
>> >                                         static methods to initialize the
>> >
>> >                             builder:
>> >
>> >                                         // Count
>> >                                         KTable<String, Long> count =
>> >
>> >                     groupedStream.count(Count.count().
>> withQueryableStoreName("my-store"));
>> >
>> >                                         // Windowed Count
>> >                                         KTable<Windowed<String>, Long>
>> >                                         windowedCount =
>> >
>> >                     groupedStream.count(Count.
>> windowed(TimeWindows.of(10L).until
>> >
>> >                 (10)).withQueryableStoreName("my-windowed-store"));
>> >
>> >                                         // Session Count
>> >                                         KTable<Windowed<String>, Long>
>> >                                         sessionCount =
>> >
>> >                     groupedStream.count(Count.sessionWindowed(
>> SessionWindows.
>> >
>> >                 with(10L)).withQueryableStoreName("my-
>> session-windowed-store"));
>> >
>> >                                     Above and below, i think i'd
>> >                                     prefer it to be:
>> >                                     groupedStream.count(/** non
>> >                                     windowed count**/)
>> >                                     groupedStream.windowed(
>> TimeWindows.of(10L)).count(...)
>> >                                     groupedStream.sessionWindowed(
>> SessionWindows.of(10L)).count(...)
>> >
>> >
>> >
>> >
>> >                                         // Reduce
>> >                                         Reducer<Long> reducer;
>> >                                         KTable<String, Long> reduce =
>> >                                         groupedStream.reduce(reducer,
>> >                                         Reduce.reduce().
>> withQueryableStoreName("my-store"));
>> >
>> >                                         // Aggregate Windowed with
>> >                                         Custom Store
>> >                                         Initializer<String> initializer;
>> >                                         Aggregator<String, Long,
>> >                                         String> aggregator;
>> >                                         KTable<Windowed<String>,
>> >                                         String> aggregate =
>> >                                         groupedStream.aggregate(
>> initializer,
>> >                                         aggregator,
>> >
>> >                     Aggregate.windowed(TimeWindows.of(10L).until(10)).
>> >
>> >                 withStateStoreSupplier(stateStoreSupplier)));
>> >
>> >                                         // Cogroup SessionWindowed
>> >                                         KTable<String, String>
>> cogrouped =
>> >
>> >                     groupedStream1.cogroup(aggregator1)
>> >
>> >
>> >                                         .cogroup(groupedStream2,
>> >                                         aggregator2)
>> >
>> >
>
>


-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Jan,

Thanks for your input. Comments inline

On Tue, 18 Jul 2017 at 15:21 Jan Filipiak <Ja...@trivago.com> wrote:

> Hi,
>
>
> 1. To many overloads:
> Currently, whenever a KTable is the result of an operation it gets and
> override with stateStoreName, and StatestoreSupplier in case people want
> to query that.
> This is what produces 2/3rd of the overloaded methods right now (not
> counting methods returning KStream)
>
>
As you state further down we are trying to address this.


> 2. Code copy and pasting.
> Almost all KTableProcessorSuppliers have the same block of (if(name !=
> null) store.put(k,v))
>
>
Yes, i agree. That is related to the KTable queryable store etc, and can
easily be addressed, but isn't necessarily part of this as it doesn't need
to be a public interface change, i.e., we can clean that up in the
background.


> 3. Runtime inefficiencies.
> Each querable table almost instantly causes a another store beeing
> required. Storing equivalent data of upstream KTables.
>

Agreed. Again, this is not a public interface change. We don't need another
store, i.e., we can just use a "View" on the existing store, which is
basically just using the KTableValueGetter to apply the map or filter
operation to the original store. We also have this jira
https://issues.apache.org/jira/browse/KAFKA-5581 to look into optimizing
when we do and don't need to add additional changelogs.


>
> So I really see us tackeling only the first part currently. Wich in my
> opinion is to short-sighted to settle on an Public API.
>

We are not settling on the public API. We do, however need to do KIPs for
public API discussions. For internal changes we don't necessarily need to
have a public discussion about it.


> This is why I want to tackle our approach to IQ-first, as it seems to me
> to be the most disruptive thing. And the cause of most problems.
>
> The Plan:
>
> Table from topic, kstream (don't even like this one, but probaly needed
> for some kind of enhanced flexibility) or aggregations would be the only
> KTables that would get associated with a statestore (their processors).
> For these operations one can have the "statestoresupplier" overload but
> also not the "querablestatestore" overload. From this point on KTables
> abstraction would be considered restored.
> All the overloads of join and through with respect to IQ would go away.
> "through" would go completely maybe the benefit added is. The method I
> would add is for a table to get a Queryhandle.
> This query handle will underneath remember its tables processor name. To
> access the data form IQ we would not rely on the "per processor
> statestore" but go the usual path through ValueGetterSupplier.
> *Note:* We do not necessarily have a Serde for V, especially after
> mapValues. also not for any intermediate Data types. It would be each
> KTableProccesors job to provide a serialized version of upstream Datatypes.
> KTableKTabkeJoinwould need to bring a JoinInputSerializer<V1,V2> that
> would serialize both upstream values for transport across boxes.
>
> This first step would kill all the "Storename" based overloads + many
> Statestore overloads. It would also avoid the bloated copy pasting in
> each KTableProcessor for maintaining the store.
> It would also make the runtime more efficient in a way that it does not
> store the same data twice, just for accessing from IQ. Tackeling problem
> 1 but also all other three problems mentioned above.
>
>  From here ~3 or 4 (from kstream,topic or aggregate) methods would still
> be stuck with StateStoresupplier overload. For me, this is quite an
> improvement already, to reduce further overloads I am thinking
> to put a nullable properties to this operations. If people want to use
> all defaults they could throw in null and it wouldn't be to painfull.
> That doesn't necessarily require
> them to have config files laying around. They could if they wanted use
> property files to create such properties + we would over to look for
> configs in the streams property.
> So the complexity of distributing property files is optional and the
> user might choose to fill the configs by code or files.
>
> I think these steps can rescue the proper abstraction of a KTable. I
> believe that with the current proposals we are only sugarcoating problem
> 1 and end up with a broken idea of what KTable is.
> I think it will be even harder to develop further from there. Interface
> wise my proposal is like developing backwards as i am very certain we
> did a wrong turn with the IQ we shouldn't try to carry through.
>
> I hope I could explain how this re factoring can tackle  the 3 above
> problems and especially why i don't think we can win tackiling only
> point 1 in the long run.
> If anything would need an implementation draft please feel free to ask
> me to provide one. Initially the proposal hopefully would get the job
> done of just removing clutter.
>
>

I agree with some of what you have said in the above few paragraphs. I
think you are correct in that KTable has become littered with a bunch of
methods to make each stage queryable, i.e, adding the overloads
for queryableStoreName and StateStoreSupplier. I think we can do away with
both of them as once you have a KTable you can always build a view of it by
using the KTableValueGetter. So we don't ever need a StateStoreSupplier as
we have one already from when the original KTable was created. We can also
possibly remove the overloads with queryableName and always use a generated
name that can be retrieved from the method `String queryableStoreName` -
this can then be used with IQ if needed.

The getQueryHandle idea you mention won't really work as things stand. The
KTable knows nothing about it's runtime context it is purely for building a
topology that can be executed. In order to successfully query a `KTable`
(state store) we need to know how many partitions and on which threads the
state stores are running. This is why we added the `stores` API to
`KafkaStreams` as this is the execution environment that has all of the
information.


Thanks,
Damian

Looking forward to your comments.
>
> Best Jan
>
>
>
> On 12.07.2017 21:27, Guozhang Wang wrote:
> > Hello Jan,
> >
> > Thanks for your feedbacks. Let me try to clarify a few things with the
> > problems that we are trying to resolve and the motivations with the
> > current proposals.
> >
> > As Matthias mentioned, one issue that we are trying to tackle is to
> > reduce the number of overloaded functions in the DSL due to serde
> > overridden / state store supplier overridden that are needed for
> > repartition, or for state store materializations. Another related
> > issue is that the current overridden state store supplier is not very
> > natural to use, for example:
> >
> > 1) If a user just want to disable caching / logging etc but do not
> > want to change the underlying store engine at all, she needs to learn
> > to know that, for example, if a windowed store or key-value store is
> > needed for this specific operator in the DSL, what serdes are needed
> > for materialize the store, in order to create a StateStoreSupplier
> > with caching / logging disabled, and then pass into the DSL.
> >
> > 2) Similarly, if a user just want to set different topic configs for
> > the changelog topic, she still need to specify the whole
> > StateStoreSupplier into the operator.
> >
> > 3) If a user want to use a different store engine (e.g. MyStore than
> > RocksDBStore) underneath but do not care about the default settings
> > for logging, caching, etc, he STILL needs to pass in the whole
> > StateStoreSupplier into the operator.
> >
> > Note that all the above scenarios are for advanced users who do want
> > to override these settings, for users who are just OK with the default
> > settings they should be not exposed with such APIs at all, like you
> > said, "I do not be exposed with any of such implementation details",
> > if you do not care.
> >
> > -----------------
> >
> > We have been talking about the configs v.s. code for such settings,
> > since we have been using configs for "global" default configs; but the
> > arguments against using configs for such per-operator / per-store
> > settings as well is that it will simply make configs hard to manage /
> > hard to wire with tools. Personally speaking, I'm not a big fan of
> > using configs for per-entity overrides and that is mainly from my
> > experience with Samza:Samza inherits exactly the same approach for
> > per-stream / per-source configs:
> >
> >
> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html
> > <
> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html>
> ([system-name][stream-id]
> > etc are all place-holders)
> >
> > The main issues were 1) users making config changes need to deploy
> > this to all the instances, I think for Streams it would be even worse
> > as we need to make a config file on each of the running instance, and
> > whenever there is a change we need to make sure they are propagated to
> > all of them, 2) whenever users make some code changes, e.g. to add a
> > new stream / system, they need to remember to set the corresponding
> > changes in the config files as well and they kept forgetting about it,
> > the lesson learned there was that it is always better to change one
> > place (code change) than two (code change + config file change).
> >
> > Again, this is not saying we have vetoed this option, and if people
> > have good reasons for this let's discuss them here.
> >
> > -----------------
> >
> > So the current proposals are mainly around keeping configs for the
> > global default settings, while still allowing users to override
> > per-operator / per-store settings in the code, while also keeping in
> > mind to not forced users to think about such implementation details if
> > they are fine with whatever the default settings. For example:
> >
> > As a normal user it is sufficient to specify an aggregation as
> >
> > ```
> > table4.join(table5, joiner).table();
> > ```
> >
> > in which she can still just focus on the computational logic with all
> > implementation details abstracted away; only if the user are capable
> > enough with the implementation details (e.g. how is the joining tables
> > be materialized into state stores, etc) and want to specify her own
> > settings (e.g. I want to swap in my own state store engine, or I want
> > to disable caching for dedup, or use a different serde etc) she can
> > "explore" them with the DSL again:
> >
> > ```
> > table4.join(table5, joiner).table(Materialized.as("store1")); // use a
> > custom store name for interactive query
> > table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier));
> > // use a custom store engine
> > table4.join(table5,
> > joiner).table(Materialized.as("store1").withLoggingEnabled(configs));
> > // use a custom store changelog topic configs
> > // ... more
> > ```
> >
> > Hope it helps.
> >
> >
> > Guozhang
> >
> >
> > On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Jan.Filipiak@trivago.com
> > <ma...@trivago.com>> wrote:
> >
> >     It makes me want to cry.
> >
> >     why on earth is the DSL going to expose all its implementation
> >     details now?
> >     especially being materialized or not.
> >
> >     If we want to take usefull steps in that direction maybe we are
> >     looking for a way to let the user switch back and forth between
> >     PAPI and DSL?
> >
> >     A change as the proposed would not eliminate any of my pain points
> >     while still being a heck of work migrating towards to.
> >
> >     Since I am only following this from the point where Eno CC'ed it
> >     into the users list:
> >
> >     Can someone please rephrase for me what problem this is trying to
> >     solve? I don't mean to be rude but It uses a problematic feature
> >     "StateStoreSuppliers in DSL" to justify making it even worse. This
> >     helps us nowhere in making the configs more flexible, its just
> >     syntactic sugar.
> >
> >     A low effort shoot like: lets add a properties to operations that
> >     would otherwise become overloaded to heavy? Or pull the configs by
> >     some naming schema
> >     form the overall properties. Additionally to that we get rid of
> >     StateStoreSuppliers in the DSL and have them also configured by
> >     said properties.
> >
> >     => way easier to migrate to, way less risk, way more flexible in
> >     the future (different implementations of the same operation don't
> >     require code change to configure)
> >
> >     Line 184 makes especially no sense to me. what is a KTableKTable
> >     non materialized join anyways?
> >
> >     Hope we can discuss more on this.
> >
> >
> >
> >
> >     On 07.07.2017 17:23, Guozhang Wang wrote:
> >
> >         I messed the indentation on github code repos; this would be
> >         easier to read:
> >
> >         https://codeshare.io/GLWW8K
> >
> >
> >         Guozhang
> >
> >
> >         On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang
> >         <wangguoz@gmail.com <ma...@gmail.com>> wrote:
> >
> >             Hi Damian / Kyle,
> >
> >             I think I agree with you guys about the pros / cons of
> >             using the builder
> >             pattern v.s. using some "secondary classes". And I'm
> >             thinking if we can
> >             take a "mid" manner between these two. I spent some time
> >             with a slight
> >             different approach from Damian's current proposal:
> >
> >
> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
> >             <
> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/>
> >             java/org/apache/kafka/streams/RefactoredAPIs.java
> >
> >             The key idea is to tolerate the final "table()" or
> >             "stream()" function to
> >             "upgrade" from the secondary classes to the first citizen
> >             classes, while
> >             having all the specs inside this function. Also this
> >             proposal includes some
> >             other refactoring that people have been discussed about
> >             for the builder to
> >             reduce the overloaded functions as well. WDYT?
> >
> >
> >             Guozhang
> >
> >
> >             On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy
> >             <damian.guy@gmail.com <ma...@gmail.com>> wrote:
> >
> >                 Hi Jan,
> >
> >                 Thanks very much for the input.
> >
> >                 On Tue, 4 Jul 2017 at 08:54 Jan Filipiak
> >                 <Jan.Filipiak@trivago.com
> >                 <ma...@trivago.com>>
> >                 wrote:
> >
> >                     Hi Damian,
> >
> >                     I do see your point of something needs to change.
> >                     But I fully agree with
> >                     Gouzhang when he says.
> >                     ---
> >
> >                     But since this is a incompatibility change, and we
> >                     are going to remove
> >
> >                 the
> >
> >                     compatibility annotations soon it means we only
> >                     have one chance and we
> >                     really have to make it right.
> >                     ----
> >
> >
> >                 I think we all agree on this one! Hence the discussion.
> >
> >
> >                     I fear all suggestions do not go far enough to
> >                     become something that
> >
> >                 will
> >
> >                     carry on for very much longer.
> >                     I am currently working on KAFKA-3705 and try to
> >                     find the most easy way
> >
> >                 for
> >
> >                     the user to give me all the required
> >                     functionality. The easiest
> >
> >                 interface I
> >
> >                     could come up so far can be looked at here.
> >
> >
> >
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
> >                     <
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
> >
> >                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
> >                 kafka/streams/kstream/internals/KTableImpl.java#L622
> >
> >
> >                 And its already horribly complicated. I am currently
> >                 unable to find the
> >
> >                     right abstraction level to have everything falling
> >                     into place
> >
> >                 naturally. To
> >
> >                     be honest I already think introducing
> >
> >
> >                 To be fair that is not a particularly easy problem to
> >                 solve!
> >
> >
> >
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
> >                     <
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
> >
> >                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
> >                 kafka/streams/kstream/internals/KTableImpl.java#L493
> >
> >                     was unideal and makes everything a mess.
> >
> >
> >                 I'm not sure i agree that it makes everything a mess,
> >                 but It could have
> >                 been done differently.
> >
> >                 The JoinType:Whatever is also not really flexible. 2
> >                 things come to my
> >                 mind:
> >
> >                     1. I don't think we should rule out config based
> >                     decisions say configs
> >
> >                 like
> >
> >
> >                      streams.$applicationID.joins.$joinname.conf = value
> >
> >                 Is this just for config? Or are you suggesting that we
> >                 could somehow
> >                 "code"
> >                 the join in a config file?
> >
> >
> >                     This can allow for tremendous changes without
> >                     single API change and IMO
> >
> >                 it
> >
> >                     was not considered enough yet.
> >
> >                     2. Push logic from the DSL to the Callback
> >                     classes. A ValueJoiner for
> >                     example can be used to implement different join
> >                     types as the user
> >
> >                 wishes.
> >                 Do you have an example of how this might look?
> >
> >
> >                     As Gouzhang said: stopping to break users is very
> >                     important.
> >
> >
> >                 Of course. We want to make it as easy as possible for
> >                 people to use
> >                 streams.
> >
> >
> >                 especially with this changes + All the plans I sadly
> >                 only have in my head
> >
> >                     but hopefully the first link can give a glimpse.
> >
> >                     Thanks for preparing the examples made it way
> >                     clearer to me what exactly
> >                     we are talking about. I would argue to go a bit
> >                     slower and more
> >
> >                 carefull on
> >
> >                     this one. At some point we need to get it right.
> >                     Peeking over to the
> >
> >                 hadoop
> >
> >                     guys with their hughe userbase. Config files
> >                     really work well for them.
> >
> >                     Best Jan
> >
> >
> >
> >
> >
> >                     On 30.06.2017 09:31, Damian Guy wrote:
> >
> >                         Thanks Matthias
> >
> >                         On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
> >                         <matthias@confluent.io
> >                         <ma...@confluent.io>>
> >
> >                     wrote:
> >
> >                             I am just catching up on this thread, so
> >                             sorry for the long email in
> >                             advance... Also, it's to some extend a
> >                             dump of thoughts and not
> >
> >                 always a
> >
> >                             clear proposal. Still need to think about
> >                             this in more detail. But
> >
> >                 maybe
> >
> >                             it helps other to get new ideas :)
> >
> >
> >                                     However, I don't understand your
> >                                     argument about putting aggregate()
> >                                     after the withXX() -- all the
> >                                     calls to withXX() set optional
> >
> >                     parameters
> >
> >                                     for aggregate() and not for
> >                                     groupBy() -- but a groupBy().withXX()
> >                                     indicates that the withXX()
> >                                     belongs to the groupBy(). IMHO, this
> >
> >                 might
> >
> >                                     be quite confusion for developers.
> >
> >
> >                                 I see what you are saying, but the
> >                                 grouped stream is effectively a
> >
> >                     no-op
> >
> >                                 until you call one of the
> >                                 aggregate/count/reduce etc functions. So
> >
> >                 the
> >
> >                                 optional params are ones that are
> >                                 applicable to any of the
> >
> >                 operations
> >
> >                     you
> >
> >                                 can perform on this grouped stream.
> >                                 Then the final
> >                                 count()/reduce()/aggregate() call has
> >                                 any of the params that are
> >                                 required/specific to that function.
> >
> >                             I understand your argument, but you don't
> >                             share the conclusion. If we
> >                             need a "final/terminal" call, the better
> >                             way might be
> >
> >                             .groupBy().count().withXX().build()
> >
> >                             (with a better name for build() though)
> >
> >
> >                         The point is that all the other calls,
> >                         i.e,withBlah, windowed, etc
> >
> >                 apply
> >
> >                         too all the aggregate functions. The terminal
> >                         call being the actual
> >
> >                 type
> >
> >                     of
> >
> >                         aggregation you want to do. I personally find
> >                         this more natural than
> >                         groupBy().count().withBlah().build()
> >
> >
> >                                 groupedStream.count(/** non windowed
> >                                 count**/)
> >
>  groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >
>  groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >
> >                             I like this. However, I don't see a reason
> >                             to have windowed() and
> >                             sessionWindowed(). We should have one
> >                             top-level `Windows` interface
> >
> >                 that
> >
> >                             both `TimeWindows` and `SessionWindows`
> >                             implement and just have a
> >
> >                 single
> >
> >                             windowed() method that accepts all
> >                             `Windows`. (I did not like the
> >                             separation of `SessionWindows` in the
> >                             first place, and this seems to
> >
> >                 be
> >
> >                             an opportunity to clean this up. It was
> >                             hard to change when we
> >                             introduced session windows)
> >
> >                         Yes - true we should look into that.
> >
> >
> >                             Btw: we do you the imperative groupBy()
> >                             and groupByKey(), and thus we
> >                             might also want to use windowBy() (instead
> >                             of windowed()). Not sure
> >
> >                 how
> >
> >                             important this is, but it seems to be
> >                             inconsistent otherwise.
> >
> >
> >                         Makes sense
> >
> >
> >                             About joins:  I don't like
> >                             .withJoinType(JoinType.LEFT) at all. I
> >
> >                 think,
> >
> >                             defining an inner/left/outer join is not
> >                             an optional argument but a
> >                             first class concept and should have a
> >                             proper representation in the
> >
> >                 API
> >
> >                             (like the current methods join(),
> >                             leftJoin, outerJoin()).
> >
> >
> >                         Yep, i did originally have it as a required
> >                         param and maybe that is
> >
> >                 what
> >
> >                     we
> >
> >                         go with. It could have a default, but maybe
> >                         that is confusing.
> >
> >
> >
> >                             About the two join API proposals, the
> >                             second one has too much boiler
> >                             plate code for my taste. Also, the actual
> >                             join() operator has only
> >
> >                 one
> >
> >                             argument what is weird to me, as in my
> >                             thinking process, the main
> >                             operator call, should have one parameter
> >                             per mandatory argument but
> >
> >                 your
> >
> >                             proposal put the mandatory arguments into
> >                             Joins.streamStreamJoin()
> >
> >                 call.
> >
> >                             This is far from intuitive IMHO.
> >
> >
> >                         This is the builder pattern, you only need one
> >                         param as the builder
> >
> >                 has
> >
> >                         captured all of the required and optional
> >                         arguments.
> >
> >
> >                             The first join proposal also seems to
> >                             align better with the pattern
> >                             suggested for aggregations and having the
> >                             same pattern for all
> >
> >                 operators
> >
> >                             is important (as you stated already).
> >
> >
> >                         This is why i offered two alternatives as i
> >                         started out with. 1 is the
> >                         builder pattern, the other is the more fluent
> >                         pattern.
> >
> >
> >                             Coming back to the config vs optional
> >                             parameter. What about having a
> >                             method withConfig[s](...) that allow to
> >                             put in the configuration?
> >
> >
> >                         Sure, it is currently called withLogConfig()
> >                         as that is the only thing
> >
> >                     that
> >
> >                         is really config.
> >
> >
> >                             This also raises the question if until()
> >                             is a windows property?
> >                             Actually, until() seems to be a
> >                             configuration parameter and thus,
> >
> >                 should
> >
> >                             not not have it's own method.
> >
> >
> >                         Hmmm, i don't agree. Until is a property of
> >                         the window. It is going
> >
> >                 to be
> >
> >                         potentially different for every window
> >                         operation you do in a streams
> >
> >                 app.
> >
> >
> >                             Browsing throw your example DSL branch, I
> >                             also saw this one:
> >
> >                                 final KTable<Windowed<String>, Long>
> >                                 windowed>
> >
> >                                groupedStream.counting()
> >
> >                                  .windowed(TimeWindows.of(10L).until(10))
> >                                                    .table();
> >
> >                             This is an interesting idea, and it remind
> >                             my on some feedback about
> >
> >                 "I
> >
> >                             wanted to count a stream, but there was no
> >                             count() method -- I first
> >                             needed to figure out, that I need to group
> >                             the stream first to be
> >
> >                 able
> >
> >                             to count it. It does make sense in
> >                             hindsight but was not obvious in
> >
> >                 the
> >
> >                             beginning". Thus, carrying out this
> >                             thought, we could also do the
> >                             following:
> >
> >
>  stream.count().groupedBy().windowedBy().table();
> >
> >                             -> Note, I use "grouped" and "windowed"
> >                             instead of imperative here,
> >
> >                 as
> >
> >                             it comes after the count()
> >
> >                             This would be more consistent than your
> >                             proposal (that has grouping
> >                             before but windowing after count()). It
> >                             might even allow us to enrich
> >                             the API with a some syntactic sugar like
> >                             `stream.count().table()` to
> >
> >                 get
> >
> >                             the overall count of all records (this
> >                             would obviously not scale,
> >
> >                 but we
> >
> >                             could support it -- if not now, maybe later).
> >
> >
> >                         I guess i'd prefer
> >                         stream.groupBy().windowBy().count()
> >                         stream.groupBy().windowBy().reduce()
> >                         stream.groupBy().count()
> >
> >                         As i said above, everything that happens
> >                         before the final aggregate
> >
> >                 call
> >
> >                         can be applied to any of them. So it makes
> >                         sense to me to do those
> >
> >                 things
> >
> >                         ahead of the final aggregate call.
> >
> >
> >                             Last about builder pattern. I am convinced
> >                             that we need some
> >
> >                 "terminal"
> >
> >                             operator/method that tells us when to add
> >                             the processor to the
> >
> >                 topology.
> >
> >                             But I don't see the need for a plain
> >                             builder pattern that feels
> >
> >                 alien to
> >
> >                             me (see my argument about the second join
> >                             proposal). Using .stream()
> >
> >                 /
> >
> >                             .table() as use in many examples might
> >                             work. But maybe a more generic
> >                             name that we can use in all places like
> >                             build() or apply() might
> >
> >                 also be
> >
> >                             an option.
> >
> >
> >                         Sure, a generic name might be ok.
> >
> >
> >
> >
> >                             -Matthias
> >
> >
> >
> >                             On 6/29/17 7:37 AM, Damian Guy wrote:
> >
> >                                 Thanks Kyle.
> >
> >                                 On Thu, 29 Jun 2017 at 15:11 Kyle
> >                                 Winkelman <
> >
> >                 winkelman.kyle@gmail.com
> >                 <ma...@gmail.com>>
> >
> >                                 wrote:
> >
> >                                     Hi Damian,
> >
> >                                                     When trying to
> >                                                     program in the
> >                                                     fluent API that
> >                                                     has been
> >
> >                 discussed
> >
> >                             most
> >
> >                                     it
> >
> >                                                     feels difficult to
> >                                                     know when you will
> >                                                     actually get an
> object
> >
> >                 you
> >
> >                     can
> >
> >                                     reuse.
> >
> >                                                     What if I make one
> >                                                     KGroupedStream
> >                                                     that I want to
> >                                                     reuse, is it
> >
> >                     legal
> >
> >                             to
> >
> >                                                     reuse it or does
> >                                                     this approach
> >                                                     expect you to call
> >                                                     grouped each
> >
> >                     time?
> >
> >                                             I'd anticipate that once
> >                                             you have a KGroupedStream
> >                                             you can
> >
> >                 re-use it
> >
> >                             as
> >
> >                                     you
> >
> >                                             can today.
> >
> >                                     You said it yourself in another
> >                                     post that the grouped stream is
> >                                     effectively a no-op until a count,
> >                                     reduce, or aggregate. The way I
> >
> >                 see
> >
> >                             it
> >
> >                                     you wouldn’t be able to reuse
> >                                     anything except KStreams and KTables,
> >
> >                             because
> >
> >                                     most of this fluent api would
> >                                     continue returning this (this being
> >
> >                 the
> >
> >                                     builder object currently being
> >                                     manipulated).
> >
> >                                 So, if you ever store a reference to
> >                                 anything but KStreams and
> >
> >                 KTables
> >
> >                             and
> >
> >                                     you use it in two different ways
> >                                     then its possible you make
> >
> >                     conflicting
> >
> >                                     withXXX() calls on the same builder.
> >
> >
> >                                 No necessarily true. It could return a
> >                                 new instance of the builder,
> >
> >                     i.e.,
> >
> >                                 the builders being immutable. So if
> >                                 you held a reference to the
> >
> >                 builder
> >
> >                             it
> >
> >                                 would always be the same as it was
> >                                 when it was created.
> >
> >
> >                                     GroupedStream<K,V>
> >                                     groupedStreamWithDefaultSerdes =
> >
> >                 kStream.grouped();
> >
> >                                     GroupedStream<K,V>
> >                                     groupedStreamWithDeclaredSerdes =
> >
>  groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >
> >                                     I’ll admit that this shouldn’t
> >                                     happen but some user is going to do
> >
> >                 it
> >
> >                                     eventually…
> >                                     Depending on implementation uses
> >                                     of groupedStreamWithDefaultSerdes
> >
> >                     would
> >
> >                                     most likely be equivalent to the
> >                                     version withDeclaredSerdes. One
> >
> >                 work
> >
> >                                     around would be to always make
> >                                     copies of the config objects you are
> >                                     building, but this approach has
> >                                     its own problem because now we
> >
> >                 have to
> >
> >                                     identify which configs are
> >                                     equivalent so we don’t create
> repeated
> >                                     processors.
> >
> >                                     The point of this long winded
> >                                     example is that we always have to be
> >                                     thinking about all of the possible
> >                                     ways it could be misused by a
> >
> >                 user
> >
> >                                     (causing them to see hard to
> >                                     diagnose problems).
> >
> >                                 Exactly! That is the point of the
> >                                 discussion really.
> >
> >
> >                                     In my attempt at a couple methods
> >                                     with builders I feel that I could
> >                                     confidently say the user couldn’t
> >                                     really mess it up.
> >
> >                                         // Count
> >                                         KTable<String, Long> count =
> >
> >
>  kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >
> >                                     The kGroupedStream is reusable and
> >                                     if they attempted to reuse the
> >
> >                     Count
> >
> >                                     for some reason it would throw an
> >                                     error message saying that a store
> >
> >                             named
> >
> >                                     “my-store” already exists.
> >
> >
> >                                 Yes i agree and i think using builders
> >                                 is my preferred pattern.
> >
> >                                 Cheers,
> >                                 Damian
> >
> >
> >                                     Thanks,
> >                                     Kyle
> >
> >                                     From: Damian Guy
> >                                     Sent: Thursday, June 29, 2017 3:59 AM
> >                                     To: dev@kafka.apache.org
> >                                     <ma...@kafka.apache.org>
> >                                     Subject: Re: [DISCUSS] Streams
> >                                     DSL/StateStore Refactoring
> >
> >                                     Hi Kyle,
> >
> >                                     Thanks for your input. Really
> >                                     appreciated.
> >
> >                                     On Thu, 29 Jun 2017 at 06:09 Kyle
> >                                     Winkelman <
> >
> >                 winkelman.kyle@gmail.com <mailto:
> winkelman.kyle@gmail.com>
> >
> >                                     wrote:
> >
> >                                         I like more of a builder
> >                                         pattern even though others
> >                                         have voiced
> >
> >                     against
> >
> >                                         it. The reason I like it is
> >                                         because it makes it clear to
> >                                         the user
> >
> >                     that
> >
> >                             a
> >
> >                                         call to KGroupedStream#count
> >                                         will return a KTable not some
> >
> >                     intermediate
> >
> >                                         class that I need to undetstand.
> >
> >                                     Yes, that makes sense.
> >
> >
> >                                         When trying to program in the
> >                                         fluent API that has been
> discussed
> >
> >                 most
> >
> >                             it
> >
> >                                         feels difficult to know when
> >                                         you will actually get an
> >                                         object you
> >
> >                 can
> >
> >                                     reuse.
> >
> >                                         What if I make one
> >                                         KGroupedStream that I want to
> >                                         reuse, is it
> >
> >                 legal
> >
> >                     to
> >
> >                                         reuse it or does this approach
> >                                         expect you to call grouped each
> >
> >                 time?
> >
> >                                     I'd anticipate that once you have
> >                                     a KGroupedStream you can re-use
> >
> >                 it
> >
> >                     as
> >
> >                             you
> >
> >                                     can today.
> >
> >
> >                                         This question doesn’t pop into
> >                                         my head at all in the builder
> >
> >                 pattern
> >
> >                     I
> >
> >                                         assume I can reuse everything.
> >                                         Finally, I like .groupByKey
> >                                         and .groupBy(KeyValueMapper)
> >                                         not a big
> >
> >                     fan
> >
> >                             of
> >
> >                                         the grouped.
> >
> >                                         Yes, grouped() was more for
> >                                         demonstration and because
> >                                         groupBy()
> >
> >                 and
> >
> >                                     groupByKey() were taken! So i'd
> >                                     imagine the api would actually
> >
> >                 want to
> >
> >                             be
> >
> >                                     groupByKey(/** no required
> >                                     args***/).withOptionalArg() and
> >                                     groupBy(KeyValueMapper
> >                                     m).withOpitionalArg(...)  of
> >                                     course this all
> >
> >                             depends
> >
> >                                     on maintaining backward
> compatibility.
> >
> >
> >                                         Unfortunately, the below
> >                                         approach would require atleast 2
> >
> >                 (probably
> >
> >                     3)
> >
> >                                         overloads (one for returning a
> >                                         KTable and one for returning a
> >
> >                 KTable
> >
> >                             with
> >
> >                                         Windowed Key, probably would
> >                                         want to split windowed and
> >
> >                     sessionwindowed
> >
> >                                     for
> >
> >                                         ease of implementation) of
> >                                         each count, reduce, and
> aggregate.
> >                                         Obviously not exhaustive but
> >                                         enough for you to get the
> picture.
> >
> >                     Count,
> >
> >                                         Reduce, and Aggregate supply 3
> >                                         static methods to initialize the
> >
> >                             builder:
> >
> >                                         // Count
> >                                         KTable<String, Long> count =
> >
> >
>  groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >
> >                                         // Windowed Count
> >                                         KTable<Windowed<String>, Long>
> >                                         windowedCount =
> >
> >
>  groupedStream.count(Count.windowed(TimeWindows.of(10L).until
> >
> >                 (10)).withQueryableStoreName("my-windowed-store"));
> >
> >                                         // Session Count
> >                                         KTable<Windowed<String>, Long>
> >                                         sessionCount =
> >
> >
>  groupedStream.count(Count.sessionWindowed(SessionWindows.
> >
> >
>  with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >
> >                                     Above and below, i think i'd
> >                                     prefer it to be:
> >                                     groupedStream.count(/** non
> >                                     windowed count**/)
> >
>  groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >
>  groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >
> >
> >
> >
> >                                         // Reduce
> >                                         Reducer<Long> reducer;
> >                                         KTable<String, Long> reduce =
> >                                         groupedStream.reduce(reducer,
> >
>  Reduce.reduce().withQueryableStoreName("my-store"));
> >
> >                                         // Aggregate Windowed with
> >                                         Custom Store
> >                                         Initializer<String> initializer;
> >                                         Aggregator<String, Long,
> >                                         String> aggregator;
> >                                         KTable<Windowed<String>,
> >                                         String> aggregate =
> >
>  groupedStream.aggregate(initializer,
> >                                         aggregator,
> >
> >                     Aggregate.windowed(TimeWindows.of(10L).until(10)).
> >
> >                 withStateStoreSupplier(stateStoreSupplier)));
> >
> >                                         // Cogroup SessionWindowed
> >                                         KTable<String, String> cogrouped
> =
> >
> >                     groupedStream1.cogroup(aggregator1)
> >
> >
> >                                         .cogroup(groupedStream2,
> >                                         aggregator2)
> >
> >

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Clemens Valiente <Cl...@trivago.com>.
Hi,

I agree with some of Jan's points here. Interactive queries are a nice to have, but not worth sacrificing clean interfaces over.
It's not the main use case of Kafka Streams and implementing it via a getQueryHandle on KTables means the related logic doesn't spread everywhere but instead truly remains optional.

Cheers
Clemens
________________________________________
From: Jan Filipiak <Ja...@trivago.com>
Sent: 18 July 2017 16:21
To: Guozhang Wang; users@kafka.apache.org
Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring

Hi,

Sorry for the delay, couldn't get to answer more early. I do understand
your point perfectly.
I just have a different perspective on what is going on. The most
crucial piece of abstraction, the KTable is falling apart
and that materializes (no pun intended) itself into many problems.

1. To many overloads:
Currently, whenever a KTable is the result of an operation it gets and
override with stateStoreName, and StatestoreSupplier in case people want
to query that.
This is what produces 2/3rd of the overloaded methods right now (not
counting methods returning KStream)

2. Code copy and pasting.
Almost all KTableProcessorSuppliers have the same block of (if(name !=
null) store.put(k,v))

3. Runtime inefficiencies.
Each querable table almost instantly causes a another store beeing
required. Storing equivalent data of upstream KTables.

So I really see us tackeling only the first part currently. Wich in my
opinion is to short-sighted to settle on an Public API.
This is why I want to tackle our approach to IQ-first, as it seems to me
to be the most disruptive thing. And the cause of most problems.

The Plan:

Table from topic, kstream (don't even like this one, but probaly needed
for some kind of enhanced flexibility) or aggregations would be the only
KTables that would get associated with a statestore (their processors).
For these operations one can have the "statestoresupplier" overload but
also not the "querablestatestore" overload. From this point on KTables
abstraction would be considered restored.
All the overloads of join and through with respect to IQ would go away.
"through" would go completely maybe the benefit added is. The method I
would add is for a table to get a Queryhandle.
This query handle will underneath remember its tables processor name. To
access the data form IQ we would not rely on the "per processor
statestore" but go the usual path through ValueGetterSupplier.
*Note:* We do not necessarily have a Serde for V, especially after
mapValues. also not for any intermediate Data types. It would be each
KTableProccesors job to provide a serialized version of upstream Datatypes.
KTableKTabkeJoinwould need to bring a JoinInputSerializer<V1,V2> that
would serialize both upstream values for transport across boxes.

This first step would kill all the "Storename" based overloads + many
Statestore overloads. It would also avoid the bloated copy pasting in
each KTableProcessor for maintaining the store.
It would also make the runtime more efficient in a way that it does not
store the same data twice, just for accessing from IQ. Tackeling problem
1 but also all other three problems mentioned above.

 From here ~3 or 4 (from kstream,topic or aggregate) methods would still
be stuck with StateStoresupplier overload. For me, this is quite an
improvement already, to reduce further overloads I am thinking
to put a nullable properties to this operations. If people want to use
all defaults they could throw in null and it wouldn't be to painfull.
That doesn't necessarily require
them to have config files laying around. They could if they wanted use
property files to create such properties + we would over to look for
configs in the streams property.
So the complexity of distributing property files is optional and the
user might choose to fill the configs by code or files.

I think these steps can rescue the proper abstraction of a KTable. I
believe that with the current proposals we are only sugarcoating problem
1 and end up with a broken idea of what KTable is.
I think it will be even harder to develop further from there. Interface
wise my proposal is like developing backwards as i am very certain we
did a wrong turn with the IQ we shouldn't try to carry through.

I hope I could explain how this re factoring can tackle  the 3 above
problems and especially why i don't think we can win tackiling only
point 1 in the long run.
If anything would need an implementation draft please feel free to ask
me to provide one. Initially the proposal hopefully would get the job
done of just removing clutter.

Looking forward to your comments.

Best Jan



On 12.07.2017 21:27, Guozhang Wang wrote:
> Hello Jan,
>
> Thanks for your feedbacks. Let me try to clarify a few things with the
> problems that we are trying to resolve and the motivations with the
> current proposals.
>
> As Matthias mentioned, one issue that we are trying to tackle is to
> reduce the number of overloaded functions in the DSL due to serde
> overridden / state store supplier overridden that are needed for
> repartition, or for state store materializations. Another related
> issue is that the current overridden state store supplier is not very
> natural to use, for example:
>
> 1) If a user just want to disable caching / logging etc but do not
> want to change the underlying store engine at all, she needs to learn
> to know that, for example, if a windowed store or key-value store is
> needed for this specific operator in the DSL, what serdes are needed
> for materialize the store, in order to create a StateStoreSupplier
> with caching / logging disabled, and then pass into the DSL.
>
> 2) Similarly, if a user just want to set different topic configs for
> the changelog topic, she still need to specify the whole
> StateStoreSupplier into the operator.
>
> 3) If a user want to use a different store engine (e.g. MyStore than
> RocksDBStore) underneath but do not care about the default settings
> for logging, caching, etc, he STILL needs to pass in the whole
> StateStoreSupplier into the operator.
>
> Note that all the above scenarios are for advanced users who do want
> to override these settings, for users who are just OK with the default
> settings they should be not exposed with such APIs at all, like you
> said, "I do not be exposed with any of such implementation details",
> if you do not care.
>
> -----------------
>
> We have been talking about the configs v.s. code for such settings,
> since we have been using configs for "global" default configs; but the
> arguments against using configs for such per-operator / per-store
> settings as well is that it will simply make configs hard to manage /
> hard to wire with tools. Personally speaking, I'm not a big fan of
> using configs for per-entity overrides and that is mainly from my
> experience with Samza:Samza inherits exactly the same approach for
> per-stream / per-source configs:
>
> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html
> <http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html> ([system-name][stream-id]
> etc are all place-holders)
>
> The main issues were 1) users making config changes need to deploy
> this to all the instances, I think for Streams it would be even worse
> as we need to make a config file on each of the running instance, and
> whenever there is a change we need to make sure they are propagated to
> all of them, 2) whenever users make some code changes, e.g. to add a
> new stream / system, they need to remember to set the corresponding
> changes in the config files as well and they kept forgetting about it,
> the lesson learned there was that it is always better to change one
> place (code change) than two (code change + config file change).
>
> Again, this is not saying we have vetoed this option, and if people
> have good reasons for this let's discuss them here.
>
> -----------------
>
> So the current proposals are mainly around keeping configs for the
> global default settings, while still allowing users to override
> per-operator / per-store settings in the code, while also keeping in
> mind to not forced users to think about such implementation details if
> they are fine with whatever the default settings. For example:
>
> As a normal user it is sufficient to specify an aggregation as
>
> ```
> table4.join(table5, joiner).table();
> ```
>
> in which she can still just focus on the computational logic with all
> implementation details abstracted away; only if the user are capable
> enough with the implementation details (e.g. how is the joining tables
> be materialized into state stores, etc) and want to specify her own
> settings (e.g. I want to swap in my own state store engine, or I want
> to disable caching for dedup, or use a different serde etc) she can
> "explore" them with the DSL again:
>
> ```
> table4.join(table5, joiner).table(Materialized.as("store1")); // use a
> custom store name for interactive query
> table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier));
> // use a custom store engine
> table4.join(table5,
> joiner).table(Materialized.as("store1").withLoggingEnabled(configs));
> // use a custom store changelog topic configs
> // ... more
> ```
>
> Hope it helps.
>
>
> Guozhang
>
>
> On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Jan.Filipiak@trivago.com
> <ma...@trivago.com>> wrote:
>
>     It makes me want to cry.
>
>     why on earth is the DSL going to expose all its implementation
>     details now?
>     especially being materialized or not.
>
>     If we want to take usefull steps in that direction maybe we are
>     looking for a way to let the user switch back and forth between
>     PAPI and DSL?
>
>     A change as the proposed would not eliminate any of my pain points
>     while still being a heck of work migrating towards to.
>
>     Since I am only following this from the point where Eno CC'ed it
>     into the users list:
>
>     Can someone please rephrase for me what problem this is trying to
>     solve? I don't mean to be rude but It uses a problematic feature
>     "StateStoreSuppliers in DSL" to justify making it even worse. This
>     helps us nowhere in making the configs more flexible, its just
>     syntactic sugar.
>
>     A low effort shoot like: lets add a properties to operations that
>     would otherwise become overloaded to heavy? Or pull the configs by
>     some naming schema
>     form the overall properties. Additionally to that we get rid of
>     StateStoreSuppliers in the DSL and have them also configured by
>     said properties.
>
>     => way easier to migrate to, way less risk, way more flexible in
>     the future (different implementations of the same operation don't
>     require code change to configure)
>
>     Line 184 makes especially no sense to me. what is a KTableKTable
>     non materialized join anyways?
>
>     Hope we can discuss more on this.
>
>
>
>
>     On 07.07.2017 17:23, Guozhang Wang wrote:
>
>         I messed the indentation on github code repos; this would be
>         easier to read:
>
>         https://codeshare.io/GLWW8K
>
>
>         Guozhang
>
>
>         On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang
>         <wangguoz@gmail.com <ma...@gmail.com>> wrote:
>
>             Hi Damian / Kyle,
>
>             I think I agree with you guys about the pros / cons of
>             using the builder
>             pattern v.s. using some "secondary classes". And I'm
>             thinking if we can
>             take a "mid" manner between these two. I spent some time
>             with a slight
>             different approach from Damian's current proposal:
>
>             https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>             <https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/>
>             java/org/apache/kafka/streams/RefactoredAPIs.java
>
>             The key idea is to tolerate the final "table()" or
>             "stream()" function to
>             "upgrade" from the secondary classes to the first citizen
>             classes, while
>             having all the specs inside this function. Also this
>             proposal includes some
>             other refactoring that people have been discussed about
>             for the builder to
>             reduce the overloaded functions as well. WDYT?
>
>
>             Guozhang
>
>
>             On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy
>             <damian.guy@gmail.com <ma...@gmail.com>> wrote:
>
>                 Hi Jan,
>
>                 Thanks very much for the input.
>
>                 On Tue, 4 Jul 2017 at 08:54 Jan Filipiak
>                 <Jan.Filipiak@trivago.com
>                 <ma...@trivago.com>>
>                 wrote:
>
>                     Hi Damian,
>
>                     I do see your point of something needs to change.
>                     But I fully agree with
>                     Gouzhang when he says.
>                     ---
>
>                     But since this is a incompatibility change, and we
>                     are going to remove
>
>                 the
>
>                     compatibility annotations soon it means we only
>                     have one chance and we
>                     really have to make it right.
>                     ----
>
>
>                 I think we all agree on this one! Hence the discussion.
>
>
>                     I fear all suggestions do not go far enough to
>                     become something that
>
>                 will
>
>                     carry on for very much longer.
>                     I am currently working on KAFKA-3705 and try to
>                     find the most easy way
>
>                 for
>
>                     the user to give me all the required
>                     functionality. The easiest
>
>                 interface I
>
>                     could come up so far can be looked at here.
>
>
>                     https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>                     <https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>
>                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>                 kafka/streams/kstream/internals/KTableImpl.java#L622
>
>
>                 And its already horribly complicated. I am currently
>                 unable to find the
>
>                     right abstraction level to have everything falling
>                     into place
>
>                 naturally. To
>
>                     be honest I already think introducing
>
>
>                 To be fair that is not a particularly easy problem to
>                 solve!
>
>
>                     https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>                     <https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>
>                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>                 kafka/streams/kstream/internals/KTableImpl.java#L493
>
>                     was unideal and makes everything a mess.
>
>
>                 I'm not sure i agree that it makes everything a mess,
>                 but It could have
>                 been done differently.
>
>                 The JoinType:Whatever is also not really flexible. 2
>                 things come to my
>                 mind:
>
>                     1. I don't think we should rule out config based
>                     decisions say configs
>
>                 like
>
>
>                      streams.$applicationID.joins.$joinname.conf = value
>
>                 Is this just for config? Or are you suggesting that we
>                 could somehow
>                 "code"
>                 the join in a config file?
>
>
>                     This can allow for tremendous changes without
>                     single API change and IMO
>
>                 it
>
>                     was not considered enough yet.
>
>                     2. Push logic from the DSL to the Callback
>                     classes. A ValueJoiner for
>                     example can be used to implement different join
>                     types as the user
>
>                 wishes.
>                 Do you have an example of how this might look?
>
>
>                     As Gouzhang said: stopping to break users is very
>                     important.
>
>
>                 Of course. We want to make it as easy as possible for
>                 people to use
>                 streams.
>
>
>                 especially with this changes + All the plans I sadly
>                 only have in my head
>
>                     but hopefully the first link can give a glimpse.
>
>                     Thanks for preparing the examples made it way
>                     clearer to me what exactly
>                     we are talking about. I would argue to go a bit
>                     slower and more
>
>                 carefull on
>
>                     this one. At some point we need to get it right.
>                     Peeking over to the
>
>                 hadoop
>
>                     guys with their hughe userbase. Config files
>                     really work well for them.
>
>                     Best Jan
>
>
>
>
>
>                     On 30.06.2017 09:31, Damian Guy wrote:
>
>                         Thanks Matthias
>
>                         On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>                         <matthias@confluent.io
>                         <ma...@confluent.io>>
>
>                     wrote:
>
>                             I am just catching up on this thread, so
>                             sorry for the long email in
>                             advance... Also, it's to some extend a
>                             dump of thoughts and not
>
>                 always a
>
>                             clear proposal. Still need to think about
>                             this in more detail. But
>
>                 maybe
>
>                             it helps other to get new ideas :)
>
>
>                                     However, I don't understand your
>                                     argument about putting aggregate()
>                                     after the withXX() -- all the
>                                     calls to withXX() set optional
>
>                     parameters
>
>                                     for aggregate() and not for
>                                     groupBy() -- but a groupBy().withXX()
>                                     indicates that the withXX()
>                                     belongs to the groupBy(). IMHO, this
>
>                 might
>
>                                     be quite confusion for developers.
>
>
>                                 I see what you are saying, but the
>                                 grouped stream is effectively a
>
>                     no-op
>
>                                 until you call one of the
>                                 aggregate/count/reduce etc functions. So
>
>                 the
>
>                                 optional params are ones that are
>                                 applicable to any of the
>
>                 operations
>
>                     you
>
>                                 can perform on this grouped stream.
>                                 Then the final
>                                 count()/reduce()/aggregate() call has
>                                 any of the params that are
>                                 required/specific to that function.
>
>                             I understand your argument, but you don't
>                             share the conclusion. If we
>                             need a "final/terminal" call, the better
>                             way might be
>
>                             .groupBy().count().withXX().build()
>
>                             (with a better name for build() though)
>
>
>                         The point is that all the other calls,
>                         i.e,withBlah, windowed, etc
>
>                 apply
>
>                         too all the aggregate functions. The terminal
>                         call being the actual
>
>                 type
>
>                     of
>
>                         aggregation you want to do. I personally find
>                         this more natural than
>                         groupBy().count().withBlah().build()
>
>
>                                 groupedStream.count(/** non windowed
>                                 count**/)
>                                 groupedStream.windowed(TimeWindows.of(10L)).count(...)
>                                 groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>                             I like this. However, I don't see a reason
>                             to have windowed() and
>                             sessionWindowed(). We should have one
>                             top-level `Windows` interface
>
>                 that
>
>                             both `TimeWindows` and `SessionWindows`
>                             implement and just have a
>
>                 single
>
>                             windowed() method that accepts all
>                             `Windows`. (I did not like the
>                             separation of `SessionWindows` in the
>                             first place, and this seems to
>
>                 be
>
>                             an opportunity to clean this up. It was
>                             hard to change when we
>                             introduced session windows)
>
>                         Yes - true we should look into that.
>
>
>                             Btw: we do you the imperative groupBy()
>                             and groupByKey(), and thus we
>                             might also want to use windowBy() (instead
>                             of windowed()). Not sure
>
>                 how
>
>                             important this is, but it seems to be
>                             inconsistent otherwise.
>
>
>                         Makes sense
>
>
>                             About joins:  I don't like
>                             .withJoinType(JoinType.LEFT) at all. I
>
>                 think,
>
>                             defining an inner/left/outer join is not
>                             an optional argument but a
>                             first class concept and should have a
>                             proper representation in the
>
>                 API
>
>                             (like the current methods join(),
>                             leftJoin, outerJoin()).
>
>
>                         Yep, i did originally have it as a required
>                         param and maybe that is
>
>                 what
>
>                     we
>
>                         go with. It could have a default, but maybe
>                         that is confusing.
>
>
>
>                             About the two join API proposals, the
>                             second one has too much boiler
>                             plate code for my taste. Also, the actual
>                             join() operator has only
>
>                 one
>
>                             argument what is weird to me, as in my
>                             thinking process, the main
>                             operator call, should have one parameter
>                             per mandatory argument but
>
>                 your
>
>                             proposal put the mandatory arguments into
>                             Joins.streamStreamJoin()
>
>                 call.
>
>                             This is far from intuitive IMHO.
>
>
>                         This is the builder pattern, you only need one
>                         param as the builder
>
>                 has
>
>                         captured all of the required and optional
>                         arguments.
>
>
>                             The first join proposal also seems to
>                             align better with the pattern
>                             suggested for aggregations and having the
>                             same pattern for all
>
>                 operators
>
>                             is important (as you stated already).
>
>
>                         This is why i offered two alternatives as i
>                         started out with. 1 is the
>                         builder pattern, the other is the more fluent
>                         pattern.
>
>
>                             Coming back to the config vs optional
>                             parameter. What about having a
>                             method withConfig[s](...) that allow to
>                             put in the configuration?
>
>
>                         Sure, it is currently called withLogConfig()
>                         as that is the only thing
>
>                     that
>
>                         is really config.
>
>
>                             This also raises the question if until()
>                             is a windows property?
>                             Actually, until() seems to be a
>                             configuration parameter and thus,
>
>                 should
>
>                             not not have it's own method.
>
>
>                         Hmmm, i don't agree. Until is a property of
>                         the window. It is going
>
>                 to be
>
>                         potentially different for every window
>                         operation you do in a streams
>
>                 app.
>
>
>                             Browsing throw your example DSL branch, I
>                             also saw this one:
>
>                                 final KTable<Windowed<String>, Long>
>                                 windowed>
>
>                                groupedStream.counting()
>
>                                  .windowed(TimeWindows.of(10L).until(10))
>                                                    .table();
>
>                             This is an interesting idea, and it remind
>                             my on some feedback about
>
>                 "I
>
>                             wanted to count a stream, but there was no
>                             count() method -- I first
>                             needed to figure out, that I need to group
>                             the stream first to be
>
>                 able
>
>                             to count it. It does make sense in
>                             hindsight but was not obvious in
>
>                 the
>
>                             beginning". Thus, carrying out this
>                             thought, we could also do the
>                             following:
>
>                             stream.count().groupedBy().windowedBy().table();
>
>                             -> Note, I use "grouped" and "windowed"
>                             instead of imperative here,
>
>                 as
>
>                             it comes after the count()
>
>                             This would be more consistent than your
>                             proposal (that has grouping
>                             before but windowing after count()). It
>                             might even allow us to enrich
>                             the API with a some syntactic sugar like
>                             `stream.count().table()` to
>
>                 get
>
>                             the overall count of all records (this
>                             would obviously not scale,
>
>                 but we
>
>                             could support it -- if not now, maybe later).
>
>
>                         I guess i'd prefer
>                         stream.groupBy().windowBy().count()
>                         stream.groupBy().windowBy().reduce()
>                         stream.groupBy().count()
>
>                         As i said above, everything that happens
>                         before the final aggregate
>
>                 call
>
>                         can be applied to any of them. So it makes
>                         sense to me to do those
>
>                 things
>
>                         ahead of the final aggregate call.
>
>
>                             Last about builder pattern. I am convinced
>                             that we need some
>
>                 "terminal"
>
>                             operator/method that tells us when to add
>                             the processor to the
>
>                 topology.
>
>                             But I don't see the need for a plain
>                             builder pattern that feels
>
>                 alien to
>
>                             me (see my argument about the second join
>                             proposal). Using .stream()
>
>                 /
>
>                             .table() as use in many examples might
>                             work. But maybe a more generic
>                             name that we can use in all places like
>                             build() or apply() might
>
>                 also be
>
>                             an option.
>
>
>                         Sure, a generic name might be ok.
>
>
>
>
>                             -Matthias
>
>
>
>                             On 6/29/17 7:37 AM, Damian Guy wrote:
>
>                                 Thanks Kyle.
>
>                                 On Thu, 29 Jun 2017 at 15:11 Kyle
>                                 Winkelman <
>
>                 winkelman.kyle@gmail.com
>                 <ma...@gmail.com>>
>
>                                 wrote:
>
>                                     Hi Damian,
>
>                                                     When trying to
>                                                     program in the
>                                                     fluent API that
>                                                     has been
>
>                 discussed
>
>                             most
>
>                                     it
>
>                                                     feels difficult to
>                                                     know when you will
>                                                     actually get an object
>
>                 you
>
>                     can
>
>                                     reuse.
>
>                                                     What if I make one
>                                                     KGroupedStream
>                                                     that I want to
>                                                     reuse, is it
>
>                     legal
>
>                             to
>
>                                                     reuse it or does
>                                                     this approach
>                                                     expect you to call
>                                                     grouped each
>
>                     time?
>
>                                             I'd anticipate that once
>                                             you have a KGroupedStream
>                                             you can
>
>                 re-use it
>
>                             as
>
>                                     you
>
>                                             can today.
>
>                                     You said it yourself in another
>                                     post that the grouped stream is
>                                     effectively a no-op until a count,
>                                     reduce, or aggregate. The way I
>
>                 see
>
>                             it
>
>                                     you wouldn't be able to reuse
>                                     anything except KStreams and KTables,
>
>                             because
>
>                                     most of this fluent api would
>                                     continue returning this (this being
>
>                 the
>
>                                     builder object currently being
>                                     manipulated).
>
>                                 So, if you ever store a reference to
>                                 anything but KStreams and
>
>                 KTables
>
>                             and
>
>                                     you use it in two different ways
>                                     then its possible you make
>
>                     conflicting
>
>                                     withXXX() calls on the same builder.
>
>
>                                 No necessarily true. It could return a
>                                 new instance of the builder,
>
>                     i.e.,
>
>                                 the builders being immutable. So if
>                                 you held a reference to the
>
>                 builder
>
>                             it
>
>                                 would always be the same as it was
>                                 when it was created.
>
>
>                                     GroupedStream<K,V>
>                                     groupedStreamWithDefaultSerdes =
>
>                 kStream.grouped();
>
>                                     GroupedStream<K,V>
>                                     groupedStreamWithDeclaredSerdes =
>                                     groupedStreamsWithDefaultSerdes.withKeySerde(.).withValueSerde(.);
>
>                                     I'll admit that this shouldn't
>                                     happen but some user is going to do
>
>                 it
>
>                                     eventually.
>                                     Depending on implementation uses
>                                     of groupedStreamWithDefaultSerdes
>
>                     would
>
>                                     most likely be equivalent to the
>                                     version withDeclaredSerdes. One
>
>                 work
>
>                                     around would be to always make
>                                     copies of the config objects you are
>                                     building, but this approach has
>                                     its own problem because now we
>
>                 have to
>
>                                     identify which configs are
>                                     equivalent so we don't create repeated
>                                     processors.
>
>                                     The point of this long winded
>                                     example is that we always have to be
>                                     thinking about all of the possible
>                                     ways it could be misused by a
>
>                 user
>
>                                     (causing them to see hard to
>                                     diagnose problems).
>
>                                 Exactly! That is the point of the
>                                 discussion really.
>
>
>                                     In my attempt at a couple methods
>                                     with builders I feel that I could
>                                     confidently say the user couldn't
>                                     really mess it up.
>
>                                         // Count
>                                         KTable<String, Long> count =
>
>                     kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
>                                     The kGroupedStream is reusable and
>                                     if they attempted to reuse the
>
>                     Count
>
>                                     for some reason it would throw an
>                                     error message saying that a store
>
>                             named
>
>                                     "my-store" already exists.
>
>
>                                 Yes i agree and i think using builders
>                                 is my preferred pattern.
>
>                                 Cheers,
>                                 Damian
>
>
>                                     Thanks,
>                                     Kyle
>
>                                     From: Damian Guy
>                                     Sent: Thursday, June 29, 2017 3:59 AM
>                                     To: dev@kafka.apache.org
>                                     <ma...@kafka.apache.org>
>                                     Subject: Re: [DISCUSS] Streams
>                                     DSL/StateStore Refactoring
>
>                                     Hi Kyle,
>
>                                     Thanks for your input. Really
>                                     appreciated.
>
>                                     On Thu, 29 Jun 2017 at 06:09 Kyle
>                                     Winkelman <
>
>                 winkelman.kyle@gmail.com <ma...@gmail.com>
>
>                                     wrote:
>
>                                         I like more of a builder
>                                         pattern even though others
>                                         have voiced
>
>                     against
>
>                                         it. The reason I like it is
>                                         because it makes it clear to
>                                         the user
>
>                     that
>
>                             a
>
>                                         call to KGroupedStream#count
>                                         will return a KTable not some
>
>                     intermediate
>
>                                         class that I need to undetstand.
>
>                                     Yes, that makes sense.
>
>
>                                         When trying to program in the
>                                         fluent API that has been discussed
>
>                 most
>
>                             it
>
>                                         feels difficult to know when
>                                         you will actually get an
>                                         object you
>
>                 can
>
>                                     reuse.
>
>                                         What if I make one
>                                         KGroupedStream that I want to
>                                         reuse, is it
>
>                 legal
>
>                     to
>
>                                         reuse it or does this approach
>                                         expect you to call grouped each
>
>                 time?
>
>                                     I'd anticipate that once you have
>                                     a KGroupedStream you can re-use
>
>                 it
>
>                     as
>
>                             you
>
>                                     can today.
>
>
>                                         This question doesn't pop into
>                                         my head at all in the builder
>
>                 pattern
>
>                     I
>
>                                         assume I can reuse everything.
>                                         Finally, I like .groupByKey
>                                         and .groupBy(KeyValueMapper)
>                                         not a big
>
>                     fan
>
>                             of
>
>                                         the grouped.
>
>                                         Yes, grouped() was more for
>                                         demonstration and because
>                                         groupBy()
>
>                 and
>
>                                     groupByKey() were taken! So i'd
>                                     imagine the api would actually
>
>                 want to
>
>                             be
>
>                                     groupByKey(/** no required
>                                     args***/).withOptionalArg() and
>                                     groupBy(KeyValueMapper
>                                     m).withOpitionalArg(...)  of
>                                     course this all
>
>                             depends
>
>                                     on maintaining backward compatibility.
>
>
>                                         Unfortunately, the below
>                                         approach would require atleast 2
>
>                 (probably
>
>                     3)
>
>                                         overloads (one for returning a
>                                         KTable and one for returning a
>
>                 KTable
>
>                             with
>
>                                         Windowed Key, probably would
>                                         want to split windowed and
>
>                     sessionwindowed
>
>                                     for
>
>                                         ease of implementation) of
>                                         each count, reduce, and aggregate.
>                                         Obviously not exhaustive but
>                                         enough for you to get the picture.
>
>                     Count,
>
>                                         Reduce, and Aggregate supply 3
>                                         static methods to initialize the
>
>                             builder:
>
>                                         // Count
>                                         KTable<String, Long> count =
>
>                     groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
>                                         // Windowed Count
>                                         KTable<Windowed<String>, Long>
>                                         windowedCount =
>
>                     groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>
>                 (10)).withQueryableStoreName("my-windowed-store"));
>
>                                         // Session Count
>                                         KTable<Windowed<String>, Long>
>                                         sessionCount =
>
>                     groupedStream.count(Count.sessionWindowed(SessionWindows.
>
>                 with(10L)).withQueryableStoreName("my-session-windowed-store"));
>
>                                     Above and below, i think i'd
>                                     prefer it to be:
>                                     groupedStream.count(/** non
>                                     windowed count**/)
>                                     groupedStream.windowed(TimeWindows.of(10L)).count(...)
>                                     groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>
>
>
>                                         // Reduce
>                                         Reducer<Long> reducer;
>                                         KTable<String, Long> reduce =
>                                         groupedStream.reduce(reducer,
>                                         Reduce.reduce().withQueryableStoreName("my-store"));
>
>                                         // Aggregate Windowed with
>                                         Custom Store
>                                         Initializer<String> initializer;
>                                         Aggregator<String, Long,
>                                         String> aggregator;
>                                         KTable<Windowed<String>,
>                                         String> aggregate =
>                                         groupedStream.aggregate(initializer,
>                                         aggregator,
>
>                     Aggregate.windowed(TimeWindows.of(10L).until(10)).
>
>                 withStateStoreSupplier(stateStoreSupplier)));
>
>                                         // Cogroup SessionWindowed
>                                         KTable<String, String> cogrouped =
>
>                     groupedStream1.cogroup(aggregator1)
>
>
>                                         .cogroup(groupedStream2,
>                                         aggregator2)
>
>                                         .aggregate(initializer,
>                                         aggregator,
>                                         Aggregate.sessionWindowed(SessionWindows.with(10L),
>                                         sessionMerger).withQueryableStoreName("my-store"));
>
>
>
>                                         public class Count {
>
>                                               public static class
>                                         Windowed extends Count {
>                                                   private Windows windows;
>                                               }
>                                               public static class
>                                         SessionWindowed extends Count {
>                                                   private
>                                         SessionWindows sessionWindows;
>                                               }
>
>                                               public static Count count();
>                                               public static Windowed
>                                         windowed(Windows windows);
>                                               public static
>                                         SessionWindowed
>                                         sessionWindowed(SessionWindows
>                                         sessionWindows);
>
>                                               // All withXXX(...) methods.
>                                         }
>
>                                         public class KGroupedStream {
>                                               public KTable<K, Long>
>                                         count(Count count);
>                                               public
>                                         KTable<Windowed<K>, Long>
>                                         count(Count.Windowed count);
>                                               public
>                                         KTable<Windowed<K>, Long>
>                                         count(Count.SessionWindowed
>
>                             count);
>
>                                         .
>                                         }
>
>
>                                         Thanks,
>                                         Kyle
>
>                                         From: Guozhang Wang
>                                         Sent: Wednesday, June 28, 2017
>                                         7:45 PM
>                                         To: dev@kafka.apache.org
>                                         <ma...@kafka.apache.org>
>                                         Subject: Re: [DISCUSS] Streams
>                                         DSL/StateStore Refactoring
>
>                                         I played the current proposal
>                                         a bit with
>
>                             https://github.com/dguy/kafka/
>
>                                         tree/dsl-experiment <
>
>                     https://github.com/dguy/kafka/tree/dsl-experiment
>                     <https://github.com/dguy/kafka/tree/dsl-experiment>
>
>                                 ,
>
>                                         and here are my observations:
>
>                                         1. Personally I prefer
>
>                                               "stream.group(mapper) /
>                                         stream.groupByKey()"
>
>                                         than
>
>
>                                         "stream.group().withKeyMapper(mapper)
>                                         / stream.group()"
>
>                                         Since 1) withKeyMapper is not
>                                         enforced programmatically
>                                         though it
>
>                 is
>
>                             not
>
>                                         "really" optional like others,
>                                         2) syntax-wise it reads more
>
>                 natural.
>
>                                         I think it is okay to add the
>                                         APIs in (
>
>
>                     https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>                     <https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr>
>
>                 c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>
>                                         )
>                                         in KGroupedStream.
>
>
>                                         2. For the
>                                         "withStateStoreSupplier" API,
>                                         are the user supposed to
>
>                     pass
>
>                             in
>
>                                         the most-inner state store
>                                         supplier (e.g. then one whose
>                                         get()
>
>                 return
>
>                                         RocksDBStore), or it is
>                                         supposed to return the
>                                         most-outer supplier
>
>                     with
>
>                                         logging / metrics / etc? I
>                                         think it would be more useful
>                                         to only
>
>                             require
>
>                                         users pass in the inner state
>                                         store supplier while specifying
>
>                     caching /
>
>                                         logging through other APIs.
>
>                                         In addition, the
>                                         "GroupedWithCustomStore" is a
>                                         bit suspicious to
>
>                 me:
>
>                     we
>
>                                     are
>
>                                         allowing users to call other
>                                         APIs like "withQueryableName"
>
>                 multiple
>
>                             time,
>
>                                         but only call
>                                         "withStateStoreSupplier" only
>                                         once in the end. Why
>
>                 is
>
>                             that?
>
>                                         3. The current DSL seems to be
>                                         only for aggregations, what about
>
>                     joins?
>
>
>                                         4. I think it is okay to keep
>                                         the "withLogConfig": for the
>                                         StateStoreSupplier it will
>                                         still be user code specifying the
>
>                 topology
>
>                             so
>
>                                     I
>
>                                         do not see there is a big
>                                         difference.
>
>
>                                         5. "WindowedGroupedStream" 's
>                                         withStateStoreSupplier should take
>
>                 the
>
>                                         windowed state store supplier
>                                         to enforce typing?
>
>
>                                         Below are minor ones:
>
>                                         6. "withQueryableName": maybe
>                                         better "withQueryableStateName"?
>
>                                         7. "withLogConfig": maybe
>                                         better "withLoggingTopicConfig()"?
>
>
>
>                                         Guozhang
>
>
>
>                                         On Wed, Jun 28, 2017 at 3:59
>                                         PM, Matthias J. Sax <
>
>                             matthias@confluent.io
>                             <ma...@confluent.io>>
>
>                                         wrote:
>
>                                             I see your point about
>                                             "when to add the processor
>                                             to the
>
>                 topology".
>
>                                     That
>
>                                             is indeed an issue. Not
>                                             sure it we could allow
>                                             "updates" to the
>
>                                         topology...
>
>                                             I don't see any problem
>                                             with having all the
>                                             withXX() in KTable
>
>                                     interface
>
>                                             -- but this might be
>                                             subjective.
>
>
>                                             However, I don't
>                                             understand your argument
>                                             about putting
>
>                 aggregate()
>
>                                             after the withXX() -- all
>                                             the calls to withXX() set
>                                             optional
>
>                             parameters
>
>                                             for aggregate() and not
>                                             for groupBy() -- but a
>                                             groupBy().withXX()
>                                             indicates that the
>                                             withXX() belongs to the
>                                             groupBy(). IMHO, this
>
>                     might
>
>                                             be quite confusion for
>                                             developers.
>
>
>                                             -Matthias
>
>                                             On 6/28/17 2:55 AM, Damian
>                                             Guy wrote:
>
>                                                     I also think that
>                                                     mixing optional
>                                                     parameters with
>                                                     configs is a
>
>                 bad
>
>                                         idea.
>
>                                                     Have not proposal
>                                                     for this atm but
>                                                     just wanted to
>                                                     mention it.
>
>                 Hope
>
>                                     to
>
>                                                     find some time to
>                                                     come up with
>                                                     something.
>
>
>                                                 Yes, i don't like the
>                                                 mix of config either.
>                                                 But the only real
>
>                     config
>
>                                         here
>
>                                                 is the logging config
>                                                 - which we don't
>                                                 really need as it can
>
>                     already
>
>                                     be
>
>                                                 done via a custom
>                                                 StateStoreSupplier.
>
>
>                                                     What I don't like
>                                                     in the current
>                                                     proposal is the
>                                                     .grouped().withKeyMapper()
>                                                     -- the current
>                                                     solution with
>
>                                     .groupBy(...)
>
>                                                     and .groupByKey()
>                                                     seems better. For
>                                                     clarity, we could
>                                                     rename to
>                                                     .groupByNewKey(...) and
>                                                     .groupByCurrentKey()
>                                                     (even if we should
>
>                     find
>
>                                                     some better names).
>
>
>                                                 it could be
>                                                 groupByKey(),
>                                                 groupBy() or something
>                                                 different bt
>
>
>
>                                                     The proposed
>                                                     pattern "chains"
>                                                     grouping and
>                                                     aggregation too
>
>                 close
>
>                                                     together. I would
>                                                     rather separate
>                                                     both more than
>                                                     less, ie, do
>
>                 into
>
>                                     the
>
>                                                     opposite direction.
>
>                                                     I am also
>                                                     wondering, if we
>                                                     could so something
>                                                     more "fluent".
>
>                 The
>
>                                         initial
>
>                                                     proposal was like:
>
>                                                             groupedStream.count()
>
>                                                              .withStoreName("name")
>                                                              .withCachingEnabled(false)
>                                                              .withLoggingEnabled(config)
>                                                                  .table()
>
>                                                     The .table()
>                                                     statement in the
>                                                     end was kinda alien.
>
>                                                 I agree, but then all
>                                                 of the withXXX methods
>                                                 need to be on
>
>                 KTable
>
>                                     which
>
>                                             is
>
>                                                 worse in my opinion.
>                                                 You also need
>                                                 something that is going to
>
>                     "build"
>
>                                         the
>
>                                                 internal processors
>                                                 and add them to the
>                                                 topology.
>
>
>                                                     The current
>                                                     proposal put the
>                                                     count() into the
>                                                     end -- ie, the
>
>                                     optional
>
>                                                     parameter for
>                                                     count() have to
>                                                     specified on the
>                                                     .grouped() call
>
>                 --
>
>                                     this
>
>                                                     does not seems to
>                                                     be the best way
>                                                     either.
>
>
>                                                 I actually prefer this
>                                                 method as you are
>                                                 building a grouped
>
>                 stream
>
>                                     that
>
>     ...
>
>     [Message clipped]
>
>
>
>
> --
> -- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi,

Sorry for the delay, couldn't get to answer more early. I do understand 
your point perfectly.
I just have a different perspective on what is going on. The most 
crucial piece of abstraction, the KTable is falling apart
and that materializes (no pun intended) itself into many problems.

1. To many overloads:
Currently, whenever a KTable is the result of an operation it gets and 
override with stateStoreName, and StatestoreSupplier in case people want 
to query that.
This is what produces 2/3rd of the overloaded methods right now (not 
counting methods returning KStream)

2. Code copy and pasting.
Almost all KTableProcessorSuppliers have the same block of (if(name != 
null) store.put(k,v))

3. Runtime inefficiencies.
Each querable table almost instantly causes a another store beeing 
required. Storing equivalent data of upstream KTables.

So I really see us tackeling only the first part currently. Wich in my 
opinion is to short-sighted to settle on an Public API.
This is why I want to tackle our approach to IQ-first, as it seems to me 
to be the most disruptive thing. And the cause of most problems.

The Plan:

Table from topic, kstream (don't even like this one, but probaly needed 
for some kind of enhanced flexibility) or aggregations would be the only 
KTables that would get associated with a statestore (their processors).
For these operations one can have the "statestoresupplier" overload but 
also not the "querablestatestore" overload. From this point on KTables 
abstraction would be considered restored.
All the overloads of join and through with respect to IQ would go away. 
"through" would go completely maybe the benefit added is. The method I 
would add is for a table to get a Queryhandle.
This query handle will underneath remember its tables processor name. To 
access the data form IQ we would not rely on the "per processor 
statestore" but go the usual path through ValueGetterSupplier.
*Note:* We do not necessarily have a Serde for V, especially after 
mapValues. also not for any intermediate Data types. It would be each 
KTableProccesors job to provide a serialized version of upstream Datatypes.
KTableKTabkeJoinwould need to bring a JoinInputSerializer<V1,V2> that 
would serialize both upstream values for transport across boxes.

This first step would kill all the "Storename" based overloads + many 
Statestore overloads. It would also avoid the bloated copy pasting in 
each KTableProcessor for maintaining the store.
It would also make the runtime more efficient in a way that it does not 
store the same data twice, just for accessing from IQ. Tackeling problem 
1 but also all other three problems mentioned above.

 From here ~3 or 4 (from kstream,topic or aggregate) methods would still 
be stuck with StateStoresupplier overload. For me, this is quite an 
improvement already, to reduce further overloads I am thinking
to put a nullable properties to this operations. If people want to use 
all defaults they could throw in null and it wouldn't be to painfull. 
That doesn't necessarily require
them to have config files laying around. They could if they wanted use 
property files to create such properties + we would over to look for 
configs in the streams property.
So the complexity of distributing property files is optional and the 
user might choose to fill the configs by code or files.

I think these steps can rescue the proper abstraction of a KTable. I 
believe that with the current proposals we are only sugarcoating problem 
1 and end up with a broken idea of what KTable is.
I think it will be even harder to develop further from there. Interface 
wise my proposal is like developing backwards as i am very certain we 
did a wrong turn with the IQ we shouldn't try to carry through.

I hope I could explain how this re factoring can tackle  the 3 above 
problems and especially why i don't think we can win tackiling only 
point 1 in the long run.
If anything would need an implementation draft please feel free to ask 
me to provide one. Initially the proposal hopefully would get the job 
done of just removing clutter.

Looking forward to your comments.

Best Jan



On 12.07.2017 21:27, Guozhang Wang wrote:
> Hello Jan,
>
> Thanks for your feedbacks. Let me try to clarify a few things with the 
> problems that we are trying to resolve and the motivations with the 
> current proposals.
>
> As Matthias mentioned, one issue that we are trying to tackle is to 
> reduce the number of overloaded functions in the DSL due to serde 
> overridden / state store supplier overridden that are needed for 
> repartition, or for state store materializations. Another related 
> issue is that the current overridden state store supplier is not very 
> natural to use, for example:
>
> 1) If a user just want to disable caching / logging etc but do not 
> want to change the underlying store engine at all, she needs to learn 
> to know that, for example, if a windowed store or key-value store is 
> needed for this specific operator in the DSL, what serdes are needed 
> for materialize the store, in order to create a StateStoreSupplier 
> with caching / logging disabled, and then pass into the DSL.
>
> 2) Similarly, if a user just want to set different topic configs for 
> the changelog topic, she still need to specify the whole 
> StateStoreSupplier into the operator.
>
> 3) If a user want to use a different store engine (e.g. MyStore than 
> RocksDBStore) underneath but do not care about the default settings 
> for logging, caching, etc, he STILL needs to pass in the whole 
> StateStoreSupplier into the operator.
>
> Note that all the above scenarios are for advanced users who do want 
> to override these settings, for users who are just OK with the default 
> settings they should be not exposed with such APIs at all, like you 
> said, "I do not be exposed with any of such implementation details", 
> if you do not care.
>
> -----------------
>
> We have been talking about the configs v.s. code for such settings, 
> since we have been using configs for "global" default configs; but the 
> arguments against using configs for such per-operator / per-store 
> settings as well is that it will simply make configs hard to manage / 
> hard to wire with tools. Personally speaking, I'm not a big fan of 
> using configs for per-entity overrides and that is mainly from my 
> experience with Samza:Samza inherits exactly the same approach for 
> per-stream / per-source configs:
>
> http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html 
> <http://samza.apache.org/learn/documentation/0.13/jobs/configuration-table.html> ([system-name][stream-id] 
> etc are all place-holders)
>
> The main issues were 1) users making config changes need to deploy 
> this to all the instances, I think for Streams it would be even worse 
> as we need to make a config file on each of the running instance, and 
> whenever there is a change we need to make sure they are propagated to 
> all of them, 2) whenever users make some code changes, e.g. to add a 
> new stream / system, they need to remember to set the corresponding 
> changes in the config files as well and they kept forgetting about it, 
> the lesson learned there was that it is always better to change one 
> place (code change) than two (code change + config file change).
>
> Again, this is not saying we have vetoed this option, and if people 
> have good reasons for this let's discuss them here.
>
> -----------------
>
> So the current proposals are mainly around keeping configs for the 
> global default settings, while still allowing users to override 
> per-operator / per-store settings in the code, while also keeping in 
> mind to not forced users to think about such implementation details if 
> they are fine with whatever the default settings. For example:
>
> As a normal user it is sufficient to specify an aggregation as
>
> ```
> table4.join(table5, joiner).table();
> ```
>
> in which she can still just focus on the computational logic with all 
> implementation details abstracted away; only if the user are capable 
> enough with the implementation details (e.g. how is the joining tables 
> be materialized into state stores, etc) and want to specify her own 
> settings (e.g. I want to swap in my own state store engine, or I want 
> to disable caching for dedup, or use a different serde etc) she can 
> "explore" them with the DSL again:
>
> ```
> table4.join(table5, joiner).table(Materialized.as("store1")); // use a 
> custom store name for interactive query
> table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier)); 
> // use a custom store engine
> table4.join(table5, 
> joiner).table(Materialized.as("store1").withLoggingEnabled(configs)); 
> // use a custom store changelog topic configs
> // ... more
> ```
>
> Hope it helps.
>
>
> Guozhang
>
>
> On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Jan.Filipiak@trivago.com 
> <ma...@trivago.com>> wrote:
>
>     It makes me want to cry.
>
>     why on earth is the DSL going to expose all its implementation
>     details now?
>     especially being materialized or not.
>
>     If we want to take usefull steps in that direction maybe we are
>     looking for a way to let the user switch back and forth between
>     PAPI and DSL?
>
>     A change as the proposed would not eliminate any of my pain points
>     while still being a heck of work migrating towards to.
>
>     Since I am only following this from the point where Eno CC'ed it
>     into the users list:
>
>     Can someone please rephrase for me what problem this is trying to
>     solve? I don't mean to be rude but It uses a problematic feature
>     "StateStoreSuppliers in DSL" to justify making it even worse. This
>     helps us nowhere in making the configs more flexible, its just
>     syntactic sugar.
>
>     A low effort shoot like: lets add a properties to operations that
>     would otherwise become overloaded to heavy? Or pull the configs by
>     some naming schema
>     form the overall properties. Additionally to that we get rid of
>     StateStoreSuppliers in the DSL and have them also configured by
>     said properties.
>
>     => way easier to migrate to, way less risk, way more flexible in
>     the future (different implementations of the same operation don't
>     require code change to configure)
>
>     Line 184 makes especially no sense to me. what is a KTableKTable
>     non materialized join anyways?
>
>     Hope we can discuss more on this.
>
>
>
>
>     On 07.07.2017 17:23, Guozhang Wang wrote:
>
>         I messed the indentation on github code repos; this would be
>         easier to read:
>
>         https://codeshare.io/GLWW8K
>
>
>         Guozhang
>
>
>         On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang
>         <wangguoz@gmail.com <ma...@gmail.com>> wrote:
>
>             Hi Damian / Kyle,
>
>             I think I agree with you guys about the pros / cons of
>             using the builder
>             pattern v.s. using some "secondary classes". And I'm
>             thinking if we can
>             take a "mid" manner between these two. I spent some time
>             with a slight
>             different approach from Damian's current proposal:
>
>             https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>             <https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/>
>             java/org/apache/kafka/streams/RefactoredAPIs.java
>
>             The key idea is to tolerate the final "table()" or
>             "stream()" function to
>             "upgrade" from the secondary classes to the first citizen
>             classes, while
>             having all the specs inside this function. Also this
>             proposal includes some
>             other refactoring that people have been discussed about
>             for the builder to
>             reduce the overloaded functions as well. WDYT?
>
>
>             Guozhang
>
>
>             On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy
>             <damian.guy@gmail.com <ma...@gmail.com>> wrote:
>
>                 Hi Jan,
>
>                 Thanks very much for the input.
>
>                 On Tue, 4 Jul 2017 at 08:54 Jan Filipiak
>                 <Jan.Filipiak@trivago.com
>                 <ma...@trivago.com>>
>                 wrote:
>
>                     Hi Damian,
>
>                     I do see your point of something needs to change.
>                     But I fully agree with
>                     Gouzhang when he says.
>                     ---
>
>                     But since this is a incompatibility change, and we
>                     are going to remove
>
>                 the
>
>                     compatibility annotations soon it means we only
>                     have one chance and we
>                     really have to make it right.
>                     ----
>
>
>                 I think we all agree on this one! Hence the discussion.
>
>
>                     I fear all suggestions do not go far enough to
>                     become something that
>
>                 will
>
>                     carry on for very much longer.
>                     I am currently working on KAFKA-3705 and try to
>                     find the most easy way
>
>                 for
>
>                     the user to give me all the required
>                     functionality. The easiest
>
>                 interface I
>
>                     could come up so far can be looked at here.
>
>
>                     https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>                     <https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>
>                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>                 kafka/streams/kstream/internals/KTableImpl.java#L622
>
>
>                 And its already horribly complicated. I am currently
>                 unable to find the
>
>                     right abstraction level to have everything falling
>                     into place
>
>                 naturally. To
>
>                     be honest I already think introducing
>
>
>                 To be fair that is not a particularly easy problem to
>                 solve!
>
>
>                     https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>                     <https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2>
>
>                 de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>                 kafka/streams/kstream/internals/KTableImpl.java#L493
>
>                     was unideal and makes everything a mess.
>
>
>                 I'm not sure i agree that it makes everything a mess,
>                 but It could have
>                 been done differently.
>
>                 The JoinType:Whatever is also not really flexible. 2
>                 things come to my
>                 mind:
>
>                     1. I don't think we should rule out config based
>                     decisions say configs
>
>                 like
>
>                            
>                      streams.$applicationID.joins.$joinname.conf = value
>
>                 Is this just for config? Or are you suggesting that we
>                 could somehow
>                 "code"
>                 the join in a config file?
>
>
>                     This can allow for tremendous changes without
>                     single API change and IMO
>
>                 it
>
>                     was not considered enough yet.
>
>                     2. Push logic from the DSL to the Callback
>                     classes. A ValueJoiner for
>                     example can be used to implement different join
>                     types as the user
>
>                 wishes.
>                 Do you have an example of how this might look?
>
>
>                     As Gouzhang said: stopping to break users is very
>                     important.
>
>
>                 Of course. We want to make it as easy as possible for
>                 people to use
>                 streams.
>
>
>                 especially with this changes + All the plans I sadly
>                 only have in my head
>
>                     but hopefully the first link can give a glimpse.
>
>                     Thanks for preparing the examples made it way
>                     clearer to me what exactly
>                     we are talking about. I would argue to go a bit
>                     slower and more
>
>                 carefull on
>
>                     this one. At some point we need to get it right.
>                     Peeking over to the
>
>                 hadoop
>
>                     guys with their hughe userbase. Config files
>                     really work well for them.
>
>                     Best Jan
>
>
>
>
>
>                     On 30.06.2017 09:31, Damian Guy wrote:
>
>                         Thanks Matthias
>
>                         On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax
>                         <matthias@confluent.io
>                         <ma...@confluent.io>>
>
>                     wrote:
>
>                             I am just catching up on this thread, so
>                             sorry for the long email in
>                             advance... Also, it's to some extend a
>                             dump of thoughts and not
>
>                 always a
>
>                             clear proposal. Still need to think about
>                             this in more detail. But
>
>                 maybe
>
>                             it helps other to get new ideas :)
>
>
>                                     However, I don't understand your
>                                     argument about putting aggregate()
>                                     after the withXX() -- all the
>                                     calls to withXX() set optional
>
>                     parameters
>
>                                     for aggregate() and not for
>                                     groupBy() -- but a groupBy().withXX()
>                                     indicates that the withXX()
>                                     belongs to the groupBy(). IMHO, this
>
>                 might
>
>                                     be quite confusion for developers.
>
>
>                                 I see what you are saying, but the
>                                 grouped stream is effectively a
>
>                     no-op
>
>                                 until you call one of the
>                                 aggregate/count/reduce etc functions. So
>
>                 the
>
>                                 optional params are ones that are
>                                 applicable to any of the
>
>                 operations
>
>                     you
>
>                                 can perform on this grouped stream.
>                                 Then the final
>                                 count()/reduce()/aggregate() call has
>                                 any of the params that are
>                                 required/specific to that function.
>
>                             I understand your argument, but you don't
>                             share the conclusion. If we
>                             need a "final/terminal" call, the better
>                             way might be
>
>                             .groupBy().count().withXX().build()
>
>                             (with a better name for build() though)
>
>
>                         The point is that all the other calls,
>                         i.e,withBlah, windowed, etc
>
>                 apply
>
>                         too all the aggregate functions. The terminal
>                         call being the actual
>
>                 type
>
>                     of
>
>                         aggregation you want to do. I personally find
>                         this more natural than
>                         groupBy().count().withBlah().build()
>
>
>                                 groupedStream.count(/** non windowed
>                                 count**/)
>                                 groupedStream.windowed(TimeWindows.of(10L)).count(...)
>                                 groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>                             I like this. However, I don't see a reason
>                             to have windowed() and
>                             sessionWindowed(). We should have one
>                             top-level `Windows` interface
>
>                 that
>
>                             both `TimeWindows` and `SessionWindows`
>                             implement and just have a
>
>                 single
>
>                             windowed() method that accepts all
>                             `Windows`. (I did not like the
>                             separation of `SessionWindows` in the
>                             first place, and this seems to
>
>                 be
>
>                             an opportunity to clean this up. It was
>                             hard to change when we
>                             introduced session windows)
>
>                         Yes - true we should look into that.
>
>
>                             Btw: we do you the imperative groupBy()
>                             and groupByKey(), and thus we
>                             might also want to use windowBy() (instead
>                             of windowed()). Not sure
>
>                 how
>
>                             important this is, but it seems to be
>                             inconsistent otherwise.
>
>
>                         Makes sense
>
>
>                             About joins:  I don't like
>                             .withJoinType(JoinType.LEFT) at all. I
>
>                 think,
>
>                             defining an inner/left/outer join is not
>                             an optional argument but a
>                             first class concept and should have a
>                             proper representation in the
>
>                 API
>
>                             (like the current methods join(),
>                             leftJoin, outerJoin()).
>
>
>                         Yep, i did originally have it as a required
>                         param and maybe that is
>
>                 what
>
>                     we
>
>                         go with. It could have a default, but maybe
>                         that is confusing.
>
>
>
>                             About the two join API proposals, the
>                             second one has too much boiler
>                             plate code for my taste. Also, the actual
>                             join() operator has only
>
>                 one
>
>                             argument what is weird to me, as in my
>                             thinking process, the main
>                             operator call, should have one parameter
>                             per mandatory argument but
>
>                 your
>
>                             proposal put the mandatory arguments into
>                             Joins.streamStreamJoin()
>
>                 call.
>
>                             This is far from intuitive IMHO.
>
>
>                         This is the builder pattern, you only need one
>                         param as the builder
>
>                 has
>
>                         captured all of the required and optional
>                         arguments.
>
>
>                             The first join proposal also seems to
>                             align better with the pattern
>                             suggested for aggregations and having the
>                             same pattern for all
>
>                 operators
>
>                             is important (as you stated already).
>
>
>                         This is why i offered two alternatives as i
>                         started out with. 1 is the
>                         builder pattern, the other is the more fluent
>                         pattern.
>
>
>                             Coming back to the config vs optional
>                             parameter. What about having a
>                             method withConfig[s](...) that allow to
>                             put in the configuration?
>
>
>                         Sure, it is currently called withLogConfig()
>                         as that is the only thing
>
>                     that
>
>                         is really config.
>
>
>                             This also raises the question if until()
>                             is a windows property?
>                             Actually, until() seems to be a
>                             configuration parameter and thus,
>
>                 should
>
>                             not not have it's own method.
>
>
>                         Hmmm, i don't agree. Until is a property of
>                         the window. It is going
>
>                 to be
>
>                         potentially different for every window
>                         operation you do in a streams
>
>                 app.
>
>
>                             Browsing throw your example DSL branch, I
>                             also saw this one:
>
>                                 final KTable<Windowed<String>, Long>
>                                 windowed>
>
>                                groupedStream.counting()
>
>                                  .windowed(TimeWindows.of(10L).until(10))
>                                                    .table();
>
>                             This is an interesting idea, and it remind
>                             my on some feedback about
>
>                 "I
>
>                             wanted to count a stream, but there was no
>                             count() method -- I first
>                             needed to figure out, that I need to group
>                             the stream first to be
>
>                 able
>
>                             to count it. It does make sense in
>                             hindsight but was not obvious in
>
>                 the
>
>                             beginning". Thus, carrying out this
>                             thought, we could also do the
>                             following:
>
>                             stream.count().groupedBy().windowedBy().table();
>
>                             -> Note, I use "grouped" and "windowed"
>                             instead of imperative here,
>
>                 as
>
>                             it comes after the count()
>
>                             This would be more consistent than your
>                             proposal (that has grouping
>                             before but windowing after count()). It
>                             might even allow us to enrich
>                             the API with a some syntactic sugar like
>                             `stream.count().table()` to
>
>                 get
>
>                             the overall count of all records (this
>                             would obviously not scale,
>
>                 but we
>
>                             could support it -- if not now, maybe later).
>
>
>                         I guess i'd prefer
>                         stream.groupBy().windowBy().count()
>                         stream.groupBy().windowBy().reduce()
>                         stream.groupBy().count()
>
>                         As i said above, everything that happens
>                         before the final aggregate
>
>                 call
>
>                         can be applied to any of them. So it makes
>                         sense to me to do those
>
>                 things
>
>                         ahead of the final aggregate call.
>
>
>                             Last about builder pattern. I am convinced
>                             that we need some
>
>                 "terminal"
>
>                             operator/method that tells us when to add
>                             the processor to the
>
>                 topology.
>
>                             But I don't see the need for a plain
>                             builder pattern that feels
>
>                 alien to
>
>                             me (see my argument about the second join
>                             proposal). Using .stream()
>
>                 /
>
>                             .table() as use in many examples might
>                             work. But maybe a more generic
>                             name that we can use in all places like
>                             build() or apply() might
>
>                 also be
>
>                             an option.
>
>
>                         Sure, a generic name might be ok.
>
>
>
>
>                             -Matthias
>
>
>
>                             On 6/29/17 7:37 AM, Damian Guy wrote:
>
>                                 Thanks Kyle.
>
>                                 On Thu, 29 Jun 2017 at 15:11 Kyle
>                                 Winkelman <
>
>                 winkelman.kyle@gmail.com
>                 <ma...@gmail.com>>
>
>                                 wrote:
>
>                                     Hi Damian,
>
>                                                     When trying to
>                                                     program in the
>                                                     fluent API that
>                                                     has been
>
>                 discussed
>
>                             most
>
>                                     it
>
>                                                     feels difficult to
>                                                     know when you will
>                                                     actually get an object
>
>                 you
>
>                     can
>
>                                     reuse.
>
>                                                     What if I make one
>                                                     KGroupedStream
>                                                     that I want to
>                                                     reuse, is it
>
>                     legal
>
>                             to
>
>                                                     reuse it or does
>                                                     this approach
>                                                     expect you to call
>                                                     grouped each
>
>                     time?
>
>                                             I'd anticipate that once
>                                             you have a KGroupedStream
>                                             you can
>
>                 re-use it
>
>                             as
>
>                                     you
>
>                                             can today.
>
>                                     You said it yourself in another
>                                     post that the grouped stream is
>                                     effectively a no-op until a count,
>                                     reduce, or aggregate. The way I
>
>                 see
>
>                             it
>
>                                     you wouldn’t be able to reuse
>                                     anything except KStreams and KTables,
>
>                             because
>
>                                     most of this fluent api would
>                                     continue returning this (this being
>
>                 the
>
>                                     builder object currently being
>                                     manipulated).
>
>                                 So, if you ever store a reference to
>                                 anything but KStreams and
>
>                 KTables
>
>                             and
>
>                                     you use it in two different ways
>                                     then its possible you make
>
>                     conflicting
>
>                                     withXXX() calls on the same builder.
>
>
>                                 No necessarily true. It could return a
>                                 new instance of the builder,
>
>                     i.e.,
>
>                                 the builders being immutable. So if
>                                 you held a reference to the
>
>                 builder
>
>                             it
>
>                                 would always be the same as it was
>                                 when it was created.
>
>
>                                     GroupedStream<K,V>
>                                     groupedStreamWithDefaultSerdes =
>
>                 kStream.grouped();
>
>                                     GroupedStream<K,V>
>                                     groupedStreamWithDeclaredSerdes =
>                                     groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>
>                                     I’ll admit that this shouldn’t
>                                     happen but some user is going to do
>
>                 it
>
>                                     eventually…
>                                     Depending on implementation uses
>                                     of groupedStreamWithDefaultSerdes
>
>                     would
>
>                                     most likely be equivalent to the
>                                     version withDeclaredSerdes. One
>
>                 work
>
>                                     around would be to always make
>                                     copies of the config objects you are
>                                     building, but this approach has
>                                     its own problem because now we
>
>                 have to
>
>                                     identify which configs are
>                                     equivalent so we don’t create repeated
>                                     processors.
>
>                                     The point of this long winded
>                                     example is that we always have to be
>                                     thinking about all of the possible
>                                     ways it could be misused by a
>
>                 user
>
>                                     (causing them to see hard to
>                                     diagnose problems).
>
>                                 Exactly! That is the point of the
>                                 discussion really.
>
>
>                                     In my attempt at a couple methods
>                                     with builders I feel that I could
>                                     confidently say the user couldn’t
>                                     really mess it up.
>
>                                         // Count
>                                         KTable<String, Long> count =
>
>                     kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
>                                     The kGroupedStream is reusable and
>                                     if they attempted to reuse the
>
>                     Count
>
>                                     for some reason it would throw an
>                                     error message saying that a store
>
>                             named
>
>                                     “my-store” already exists.
>
>
>                                 Yes i agree and i think using builders
>                                 is my preferred pattern.
>
>                                 Cheers,
>                                 Damian
>
>
>                                     Thanks,
>                                     Kyle
>
>                                     From: Damian Guy
>                                     Sent: Thursday, June 29, 2017 3:59 AM
>                                     To: dev@kafka.apache.org
>                                     <ma...@kafka.apache.org>
>                                     Subject: Re: [DISCUSS] Streams
>                                     DSL/StateStore Refactoring
>
>                                     Hi Kyle,
>
>                                     Thanks for your input. Really
>                                     appreciated.
>
>                                     On Thu, 29 Jun 2017 at 06:09 Kyle
>                                     Winkelman <
>
>                 winkelman.kyle@gmail.com <ma...@gmail.com>
>
>                                     wrote:
>
>                                         I like more of a builder
>                                         pattern even though others
>                                         have voiced
>
>                     against
>
>                                         it. The reason I like it is
>                                         because it makes it clear to
>                                         the user
>
>                     that
>
>                             a
>
>                                         call to KGroupedStream#count
>                                         will return a KTable not some
>
>                     intermediate
>
>                                         class that I need to undetstand.
>
>                                     Yes, that makes sense.
>
>
>                                         When trying to program in the
>                                         fluent API that has been discussed
>
>                 most
>
>                             it
>
>                                         feels difficult to know when
>                                         you will actually get an
>                                         object you
>
>                 can
>
>                                     reuse.
>
>                                         What if I make one
>                                         KGroupedStream that I want to
>                                         reuse, is it
>
>                 legal
>
>                     to
>
>                                         reuse it or does this approach
>                                         expect you to call grouped each
>
>                 time?
>
>                                     I'd anticipate that once you have
>                                     a KGroupedStream you can re-use
>
>                 it
>
>                     as
>
>                             you
>
>                                     can today.
>
>
>                                         This question doesn’t pop into
>                                         my head at all in the builder
>
>                 pattern
>
>                     I
>
>                                         assume I can reuse everything.
>                                         Finally, I like .groupByKey
>                                         and .groupBy(KeyValueMapper)
>                                         not a big
>
>                     fan
>
>                             of
>
>                                         the grouped.
>
>                                         Yes, grouped() was more for
>                                         demonstration and because
>                                         groupBy()
>
>                 and
>
>                                     groupByKey() were taken! So i'd
>                                     imagine the api would actually
>
>                 want to
>
>                             be
>
>                                     groupByKey(/** no required
>                                     args***/).withOptionalArg() and
>                                     groupBy(KeyValueMapper
>                                     m).withOpitionalArg(...)  of
>                                     course this all
>
>                             depends
>
>                                     on maintaining backward compatibility.
>
>
>                                         Unfortunately, the below
>                                         approach would require atleast 2
>
>                 (probably
>
>                     3)
>
>                                         overloads (one for returning a
>                                         KTable and one for returning a
>
>                 KTable
>
>                             with
>
>                                         Windowed Key, probably would
>                                         want to split windowed and
>
>                     sessionwindowed
>
>                                     for
>
>                                         ease of implementation) of
>                                         each count, reduce, and aggregate.
>                                         Obviously not exhaustive but
>                                         enough for you to get the picture.
>
>                     Count,
>
>                                         Reduce, and Aggregate supply 3
>                                         static methods to initialize the
>
>                             builder:
>
>                                         // Count
>                                         KTable<String, Long> count =
>
>                     groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
>                                         // Windowed Count
>                                         KTable<Windowed<String>, Long>
>                                         windowedCount =
>
>                     groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>
>                 (10)).withQueryableStoreName("my-windowed-store"));
>
>                                         // Session Count
>                                         KTable<Windowed<String>, Long>
>                                         sessionCount =
>
>                     groupedStream.count(Count.sessionWindowed(SessionWindows.
>
>                 with(10L)).withQueryableStoreName("my-session-windowed-store"));
>
>                                     Above and below, i think i'd
>                                     prefer it to be:
>                                     groupedStream.count(/** non
>                                     windowed count**/)
>                                     groupedStream.windowed(TimeWindows.of(10L)).count(...)
>                                     groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>
>
>
>                                         // Reduce
>                                         Reducer<Long> reducer;
>                                         KTable<String, Long> reduce =
>                                         groupedStream.reduce(reducer,
>                                         Reduce.reduce().withQueryableStoreName("my-store"));
>
>                                         // Aggregate Windowed with
>                                         Custom Store
>                                         Initializer<String> initializer;
>                                         Aggregator<String, Long,
>                                         String> aggregator;
>                                         KTable<Windowed<String>,
>                                         String> aggregate =
>                                         groupedStream.aggregate(initializer,
>                                         aggregator,
>
>                     Aggregate.windowed(TimeWindows.of(10L).until(10)).
>
>                 withStateStoreSupplier(stateStoreSupplier)));
>
>                                         // Cogroup SessionWindowed
>                                         KTable<String, String> cogrouped =
>
>                     groupedStream1.cogroup(aggregator1)
>
>                                                  
>                                         .cogroup(groupedStream2,
>                                         aggregator2)
>                                                  
>                                         .aggregate(initializer,
>                                         aggregator,
>                                         Aggregate.sessionWindowed(SessionWindows.with(10L),
>                                         sessionMerger).withQueryableStoreName("my-store"));
>
>
>
>                                         public class Count {
>
>                                               public static class
>                                         Windowed extends Count {
>                                                   private Windows windows;
>                                               }
>                                               public static class
>                                         SessionWindowed extends Count {
>                                                   private
>                                         SessionWindows sessionWindows;
>                                               }
>
>                                               public static Count count();
>                                               public static Windowed
>                                         windowed(Windows windows);
>                                               public static
>                                         SessionWindowed
>                                         sessionWindowed(SessionWindows
>                                         sessionWindows);
>
>                                               // All withXXX(...) methods.
>                                         }
>
>                                         public class KGroupedStream {
>                                               public KTable<K, Long>
>                                         count(Count count);
>                                               public
>                                         KTable<Windowed<K>, Long>
>                                         count(Count.Windowed count);
>                                               public
>                                         KTable<Windowed<K>, Long>
>                                         count(Count.SessionWindowed
>
>                             count);
>
>                                         …
>                                         }
>
>
>                                         Thanks,
>                                         Kyle
>
>                                         From: Guozhang Wang
>                                         Sent: Wednesday, June 28, 2017
>                                         7:45 PM
>                                         To: dev@kafka.apache.org
>                                         <ma...@kafka.apache.org>
>                                         Subject: Re: [DISCUSS] Streams
>                                         DSL/StateStore Refactoring
>
>                                         I played the current proposal
>                                         a bit with
>
>                             https://github.com/dguy/kafka/
>
>                                         tree/dsl-experiment <
>
>                     https://github.com/dguy/kafka/tree/dsl-experiment
>                     <https://github.com/dguy/kafka/tree/dsl-experiment>
>
>                                 ,
>
>                                         and here are my observations:
>
>                                         1. Personally I prefer
>
>                                               "stream.group(mapper) /
>                                         stream.groupByKey()"
>
>                                         than
>
>                                              
>                                         "stream.group().withKeyMapper(mapper)
>                                         / stream.group()"
>
>                                         Since 1) withKeyMapper is not
>                                         enforced programmatically
>                                         though it
>
>                 is
>
>                             not
>
>                                         "really" optional like others,
>                                         2) syntax-wise it reads more
>
>                 natural.
>
>                                         I think it is okay to add the
>                                         APIs in (
>
>
>                     https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>                     <https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr>
>
>                 c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>
>                                         )
>                                         in KGroupedStream.
>
>
>                                         2. For the
>                                         "withStateStoreSupplier" API,
>                                         are the user supposed to
>
>                     pass
>
>                             in
>
>                                         the most-inner state store
>                                         supplier (e.g. then one whose
>                                         get()
>
>                 return
>
>                                         RocksDBStore), or it is
>                                         supposed to return the
>                                         most-outer supplier
>
>                     with
>
>                                         logging / metrics / etc? I
>                                         think it would be more useful
>                                         to only
>
>                             require
>
>                                         users pass in the inner state
>                                         store supplier while specifying
>
>                     caching /
>
>                                         logging through other APIs.
>
>                                         In addition, the
>                                         "GroupedWithCustomStore" is a
>                                         bit suspicious to
>
>                 me:
>
>                     we
>
>                                     are
>
>                                         allowing users to call other
>                                         APIs like "withQueryableName"
>
>                 multiple
>
>                             time,
>
>                                         but only call
>                                         "withStateStoreSupplier" only
>                                         once in the end. Why
>
>                 is
>
>                             that?
>
>                                         3. The current DSL seems to be
>                                         only for aggregations, what about
>
>                     joins?
>
>
>                                         4. I think it is okay to keep
>                                         the "withLogConfig": for the
>                                         StateStoreSupplier it will
>                                         still be user code specifying the
>
>                 topology
>
>                             so
>
>                                     I
>
>                                         do not see there is a big
>                                         difference.
>
>
>                                         5. "WindowedGroupedStream" 's
>                                         withStateStoreSupplier should take
>
>                 the
>
>                                         windowed state store supplier
>                                         to enforce typing?
>
>
>                                         Below are minor ones:
>
>                                         6. "withQueryableName": maybe
>                                         better "withQueryableStateName"?
>
>                                         7. "withLogConfig": maybe
>                                         better "withLoggingTopicConfig()"?
>
>
>
>                                         Guozhang
>
>
>
>                                         On Wed, Jun 28, 2017 at 3:59
>                                         PM, Matthias J. Sax <
>
>                             matthias@confluent.io
>                             <ma...@confluent.io>>
>
>                                         wrote:
>
>                                             I see your point about
>                                             "when to add the processor
>                                             to the
>
>                 topology".
>
>                                     That
>
>                                             is indeed an issue. Not
>                                             sure it we could allow
>                                             "updates" to the
>
>                                         topology...
>
>                                             I don't see any problem
>                                             with having all the
>                                             withXX() in KTable
>
>                                     interface
>
>                                             -- but this might be
>                                             subjective.
>
>
>                                             However, I don't
>                                             understand your argument
>                                             about putting
>
>                 aggregate()
>
>                                             after the withXX() -- all
>                                             the calls to withXX() set
>                                             optional
>
>                             parameters
>
>                                             for aggregate() and not
>                                             for groupBy() -- but a
>                                             groupBy().withXX()
>                                             indicates that the
>                                             withXX() belongs to the
>                                             groupBy(). IMHO, this
>
>                     might
>
>                                             be quite confusion for
>                                             developers.
>
>
>                                             -Matthias
>
>                                             On 6/28/17 2:55 AM, Damian
>                                             Guy wrote:
>
>                                                     I also think that
>                                                     mixing optional
>                                                     parameters with
>                                                     configs is a
>
>                 bad
>
>                                         idea.
>
>                                                     Have not proposal
>                                                     for this atm but
>                                                     just wanted to
>                                                     mention it.
>
>                 Hope
>
>                                     to
>
>                                                     find some time to
>                                                     come up with
>                                                     something.
>
>
>                                                 Yes, i don't like the
>                                                 mix of config either.
>                                                 But the only real
>
>                     config
>
>                                         here
>
>                                                 is the logging config
>                                                 - which we don't
>                                                 really need as it can
>
>                     already
>
>                                     be
>
>                                                 done via a custom
>                                                 StateStoreSupplier.
>
>
>                                                     What I don't like
>                                                     in the current
>                                                     proposal is the
>                                                     .grouped().withKeyMapper()
>                                                     -- the current
>                                                     solution with
>
>                                     .groupBy(...)
>
>                                                     and .groupByKey()
>                                                     seems better. For
>                                                     clarity, we could
>                                                     rename to
>                                                     .groupByNewKey(...) and
>                                                     .groupByCurrentKey()
>                                                     (even if we should
>
>                     find
>
>                                                     some better names).
>
>
>                                                 it could be
>                                                 groupByKey(),
>                                                 groupBy() or something
>                                                 different bt
>
>
>
>                                                     The proposed
>                                                     pattern "chains"
>                                                     grouping and
>                                                     aggregation too
>
>                 close
>
>                                                     together. I would
>                                                     rather separate
>                                                     both more than
>                                                     less, ie, do
>
>                 into
>
>                                     the
>
>                                                     opposite direction.
>
>                                                     I am also
>                                                     wondering, if we
>                                                     could so something
>                                                     more "fluent".
>
>                 The
>
>                                         initial
>
>                                                     proposal was like:
>
>                                                             groupedStream.count()
>                                                                
>                                                              .withStoreName("name")
>                                                              .withCachingEnabled(false)
>                                                              .withLoggingEnabled(config)
>                                                                  .table()
>
>                                                     The .table()
>                                                     statement in the
>                                                     end was kinda alien.
>
>                                                 I agree, but then all
>                                                 of the withXXX methods
>                                                 need to be on
>
>                 KTable
>
>                                     which
>
>                                             is
>
>                                                 worse in my opinion.
>                                                 You also need
>                                                 something that is going to
>
>                     "build"
>
>                                         the
>
>                                                 internal processors
>                                                 and add them to the
>                                                 topology.
>
>
>                                                     The current
>                                                     proposal put the
>                                                     count() into the
>                                                     end -- ie, the
>
>                                     optional
>
>                                                     parameter for
>                                                     count() have to
>                                                     specified on the
>                                                     .grouped() call
>
>                 --
>
>                                     this
>
>                                                     does not seems to
>                                                     be the best way
>                                                     either.
>
>
>                                                 I actually prefer this
>                                                 method as you are
>                                                 building a grouped
>
>                 stream
>
>                                     that
>
>     ...
>
>     [Message clipped] 
>
>
>
>
> -- 
> -- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Jan,

Thanks for your feedbacks. Let me try to clarify a few things with the
problems that we are trying to resolve and the motivations with the current
proposals.

As Matthias mentioned, one issue that we are trying to tackle is to reduce
the number of overloaded functions in the DSL due to serde overridden /
state store supplier overridden that are needed for repartition, or for
state store materializations. Another related issue is that the current
overridden state store supplier is not very natural to use, for example:

1) If a user just want to disable caching / logging etc but do not want to
change the underlying store engine at all, she needs to learn to know that,
for example, if a windowed store or key-value store is needed for this
specific operator in the DSL, what serdes are needed for materialize the
store, in order to create a StateStoreSupplier with caching / logging
disabled, and then pass into the DSL.

2) Similarly, if a user just want to set different topic configs for the
changelog topic, she still need to specify the whole StateStoreSupplier
into the operator.

3) If a user want to use a different store engine (e.g. MyStore than
RocksDBStore) underneath but do not care about the default settings for
logging, caching, etc, he STILL needs to pass in the whole
StateStoreSupplier into the operator.

Note that all the above scenarios are for advanced users who do want to
override these settings, for users who are just OK with the default
settings they should be not exposed with such APIs at all, like you said,
"I do not be exposed with any of such implementation details", if you do
not care.

-----------------

We have been talking about the configs v.s. code for such settings, since
we have been using configs for "global" default configs; but the arguments
against using configs for such per-operator / per-store settings as well is
that it will simply make configs hard to manage / hard to wire with tools.
Personally speaking, I'm not a big fan of using configs for per-entity
overrides and that is mainly from my experience with Samza: Samza inherits
exactly the same approach for per-stream / per-source configs:

http://samza.apache.org/learn/documentation/0.13/jobs/configuration
-table.html ([system-name][stream-id] etc are all place-holders)

The main issues were 1) users making config changes need to deploy this to
all the instances, I think for Streams it would be even worse as we need to
make a config file on each of the running instance, and whenever there is a
change we need to make sure they are propagated to all of them, 2) whenever
users make some code changes, e.g. to add a new stream / system, they need
to remember to set the corresponding changes in the config files as well
and they kept forgetting about it, the lesson learned there was that it is
always better to change one place (code change) than two (code change +
config file change).

Again, this is not saying we have vetoed this option, and if people have
good reasons for this let's discuss them here.

-----------------

So the current proposals are mainly around keeping configs for the global
default settings, while still allowing users to override per-operator /
per-store settings in the code, while also keeping in mind to not forced
users to think about such implementation details if they are fine with
whatever the default settings. For example:

As a normal user it is sufficient to specify an aggregation as

```
table4.join(table5, joiner).table();
```

in which she can still just focus on the computational logic with all
implementation details abstracted away; only if the user are capable enough
with the implementation details (e.g. how is the joining tables be
materialized into state stores, etc) and want to specify her own settings
(e.g. I want to swap in my own state store engine, or I want to disable
caching for dedup, or use a different serde etc) she can "explore" them
with the DSL again:

```
table4.join(table5, joiner).table(Materialized.as("store1")); // use a
custom store name for interactive query
table4.join(table5, joiner).table(Materialized.as(MyStoreSupplier)); // use
a custom store engine
table4.join(table5, joiner).table(Materialized.as("store1"
).withLoggingEnabled(configs)); // use a custom store changelog topic
configs
// ... more
```

Hope it helps.


Guozhang


On Fri, Jul 7, 2017 at 3:42 PM, Jan Filipiak <Ja...@trivago.com>
wrote:

> It makes me want to cry.
>
> why on earth is the DSL going to expose all its implementation details now?
> especially being materialized or not.
>
> If we want to take usefull steps in that direction maybe we are looking
> for a way to let the user switch back and forth between PAPI and DSL?
>
> A change as the proposed would not eliminate any of my pain points while
> still being a heck of work migrating towards to.
>
> Since I am only following this from the point where Eno CC'ed it into the
> users list:
>
> Can someone please rephrase for me what problem this is trying to solve? I
> don't mean to be rude but It uses a problematic feature
> "StateStoreSuppliers in DSL" to justify making it even worse. This helps
> us nowhere in making the configs more flexible, its just syntactic sugar.
>
> A low effort shoot like: lets add a properties to operations that would
> otherwise become overloaded to heavy? Or pull the configs by some naming
> schema
> form the overall properties. Additionally to that we get rid of
> StateStoreSuppliers in the DSL and have them also configured by said
> properties.
>
> => way easier to migrate to, way less risk, way more flexible in the
> future (different implementations of the same operation don't require code
> change to configure)
>
> Line 184 makes especially no sense to me. what is a KTableKTable non
> materialized join anyways?
>
> Hope we can discuss more on this.
>
>
>
>
> On 07.07.2017 17:23, Guozhang Wang wrote:
>
>> I messed the indentation on github code repos; this would be easier to
>> read:
>>
>> https://codeshare.io/GLWW8K
>>
>>
>> Guozhang
>>
>>
>> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>> Hi Damian / Kyle,
>>>
>>> I think I agree with you guys about the pros / cons of using the builder
>>> pattern v.s. using some "secondary classes". And I'm thinking if we can
>>> take a "mid" manner between these two. I spent some time with a slight
>>> different approach from Damian's current proposal:
>>>
>>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/
>>> streams/src/main/
>>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>>
>>> The key idea is to tolerate the final "table()" or "stream()" function to
>>> "upgrade" from the secondary classes to the first citizen classes, while
>>> having all the specs inside this function. Also this proposal includes
>>> some
>>> other refactoring that people have been discussed about for the builder
>>> to
>>> reduce the overloaded functions as well. WDYT?
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>>>
>>> Hi Jan,
>>>>
>>>> Thanks very much for the input.
>>>>
>>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>>> wrote:
>>>>
>>>> Hi Damian,
>>>>>
>>>>> I do see your point of something needs to change. But I fully agree
>>>>> with
>>>>> Gouzhang when he says.
>>>>> ---
>>>>>
>>>>> But since this is a incompatibility change, and we are going to remove
>>>>>
>>>> the
>>>>
>>>>> compatibility annotations soon it means we only have one chance and we
>>>>> really have to make it right.
>>>>> ----
>>>>>
>>>>>
>>>>> I think we all agree on this one! Hence the discussion.
>>>>
>>>>
>>>> I fear all suggestions do not go far enough to become something that
>>>>>
>>>> will
>>>>
>>>>> carry on for very much longer.
>>>>> I am currently working on KAFKA-3705 and try to find the most easy way
>>>>>
>>>> for
>>>>
>>>>> the user to give me all the required functionality. The easiest
>>>>>
>>>> interface I
>>>>
>>>>> could come up so far can be looked at here.
>>>>>
>>>>>
>>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>
>>>>>
>>>>> And its already horribly complicated. I am currently unable to find the
>>>>
>>>>> right abstraction level to have everything falling into place
>>>>>
>>>> naturally. To
>>>>
>>>>> be honest I already think introducing
>>>>>
>>>>>
>>>>> To be fair that is not a particularly easy problem to solve!
>>>>
>>>>
>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>>>>
>>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>>
>>>>> was unideal and makes everything a mess.
>>>>>
>>>>
>>>> I'm not sure i agree that it makes everything a mess, but It could have
>>>> been done differently.
>>>>
>>>> The JoinType:Whatever is also not really flexible. 2 things come to my
>>>> mind:
>>>>
>>>>> 1. I don't think we should rule out config based decisions say configs
>>>>>
>>>> like
>>>>
>>>>>          streams.$applicationID.joins.$joinname.conf = value
>>>>>
>>>>> Is this just for config? Or are you suggesting that we could somehow
>>>> "code"
>>>> the join in a config file?
>>>>
>>>>
>>>> This can allow for tremendous changes without single API change and IMO
>>>>>
>>>> it
>>>>
>>>>> was not considered enough yet.
>>>>>
>>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>>>>> example can be used to implement different join types as the user
>>>>>
>>>> wishes.
>>>> Do you have an example of how this might look?
>>>>
>>>>
>>>> As Gouzhang said: stopping to break users is very important.
>>>>>
>>>>
>>>> Of course. We want to make it as easy as possible for people to use
>>>> streams.
>>>>
>>>>
>>>> especially with this changes + All the plans I sadly only have in my
>>>> head
>>>>
>>>>> but hopefully the first link can give a glimpse.
>>>>>
>>>>> Thanks for preparing the examples made it way clearer to me what
>>>>> exactly
>>>>> we are talking about. I would argue to go a bit slower and more
>>>>>
>>>> carefull on
>>>>
>>>>> this one. At some point we need to get it right. Peeking over to the
>>>>>
>>>> hadoop
>>>>
>>>>> guys with their hughe userbase. Config files really work well for them.
>>>>>
>>>>> Best Jan
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>>
>>>>>> Thanks Matthias
>>>>>>
>>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>>>>>>
>>>>> wrote:
>>>>>
>>>>>> I am just catching up on this thread, so sorry for the long email in
>>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>>>>>>
>>>>>> always a
>>>>
>>>>> clear proposal. Still need to think about this in more detail. But
>>>>>>>
>>>>>> maybe
>>>>
>>>>> it helps other to get new ideas :)
>>>>>>>
>>>>>>>
>>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>>>
>>>>>>>> parameters
>>>>>
>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>>>>>
>>>>>>>> might
>>>>
>>>>> be quite confusion for developers.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I see what you are saying, but the grouped stream is effectively a
>>>>>>>>
>>>>>>> no-op
>>>>>
>>>>>> until you call one of the aggregate/count/reduce etc functions. So
>>>>>>>>
>>>>>>> the
>>>>
>>>>> optional params are ones that are applicable to any of the
>>>>>>>>
>>>>>>> operations
>>>>
>>>>> you
>>>>>
>>>>>> can perform on this grouped stream. Then the final
>>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>>> required/specific to that function.
>>>>>>>>
>>>>>>>> I understand your argument, but you don't share the conclusion. If
>>>>>>> we
>>>>>>> need a "final/terminal" call, the better way might be
>>>>>>>
>>>>>>> .groupBy().count().withXX().build()
>>>>>>>
>>>>>>> (with a better name for build() though)
>>>>>>>
>>>>>>>
>>>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>>>>>
>>>>> apply
>>>>
>>>>> too all the aggregate functions. The terminal call being the actual
>>>>>>
>>>>> type
>>>>
>>>>> of
>>>>>
>>>>>> aggregation you want to do. I personally find this more natural than
>>>>>> groupBy().count().withBlah().build()
>>>>>>
>>>>>>
>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>
>>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>>> sessionWindowed(). We should have one top-level `Windows` interface
>>>>>>>
>>>>>> that
>>>>
>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>>>>>>
>>>>>> single
>>>>
>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>>> separation of `SessionWindows` in the first place, and this seems to
>>>>>>>
>>>>>> be
>>>>
>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>>> introduced session windows)
>>>>>>>
>>>>>>> Yes - true we should look into that.
>>>>>>
>>>>>>
>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>>>>>>> might also want to use windowBy() (instead of windowed()). Not sure
>>>>>>>
>>>>>> how
>>>>
>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>>
>>>>>>>
>>>>>>> Makes sense
>>>>>>
>>>>>>
>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>>>>>>
>>>>>> think,
>>>>
>>>>> defining an inner/left/outer join is not an optional argument but a
>>>>>>> first class concept and should have a proper representation in the
>>>>>>>
>>>>>> API
>>>>
>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>>
>>>>>>>
>>>>>>> Yep, i did originally have it as a required param and maybe that is
>>>>>>
>>>>> what
>>>>
>>>>> we
>>>>>
>>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>>
>>>>>>
>>>>>>
>>>>>> About the two join API proposals, the second one has too much boiler
>>>>>>> plate code for my taste. Also, the actual join() operator has only
>>>>>>>
>>>>>> one
>>>>
>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>>> operator call, should have one parameter per mandatory argument but
>>>>>>>
>>>>>> your
>>>>
>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>>>>>>
>>>>>> call.
>>>>
>>>>> This is far from intuitive IMHO.
>>>>>>>
>>>>>>>
>>>>>>> This is the builder pattern, you only need one param as the builder
>>>>>>
>>>>> has
>>>>
>>>>> captured all of the required and optional arguments.
>>>>>>
>>>>>>
>>>>>> The first join proposal also seems to align better with the pattern
>>>>>>> suggested for aggregations and having the same pattern for all
>>>>>>>
>>>>>> operators
>>>>
>>>>> is important (as you stated already).
>>>>>>>
>>>>>>>
>>>>>>> This is why i offered two alternatives as i started out with. 1 is
>>>>>> the
>>>>>> builder pattern, the other is the more fluent pattern.
>>>>>>
>>>>>>
>>>>>> Coming back to the config vs optional parameter. What about having a
>>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>>
>>>>>>>
>>>>>>> Sure, it is currently called withLogConfig() as that is the only
>>>>>> thing
>>>>>>
>>>>> that
>>>>>
>>>>>> is really config.
>>>>>>
>>>>>>
>>>>>> This also raises the question if until() is a windows property?
>>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>>>>>>
>>>>>> should
>>>>
>>>>> not not have it's own method.
>>>>>>>
>>>>>>>
>>>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>>>>>
>>>>> to be
>>>>
>>>>> potentially different for every window operation you do in a streams
>>>>>>
>>>>> app.
>>>>
>>>>>
>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>>
>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>>>
>>>>>>>    groupedStream.counting()
>>>>>>>
>>>>>>>>                    .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>                    .table();
>>>>>>>>
>>>>>>> This is an interesting idea, and it remind my on some feedback about
>>>>>>>
>>>>>> "I
>>>>
>>>>> wanted to count a stream, but there was no count() method -- I first
>>>>>>> needed to figure out, that I need to group the stream first to be
>>>>>>>
>>>>>> able
>>>>
>>>>> to count it. It does make sense in hindsight but was not obvious in
>>>>>>>
>>>>>> the
>>>>
>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>>> following:
>>>>>>>
>>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>>
>>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative here,
>>>>>>>
>>>>>> as
>>>>
>>>>> it comes after the count()
>>>>>>>
>>>>>>> This would be more consistent than your proposal (that has grouping
>>>>>>> before but windowing after count()). It might even allow us to enrich
>>>>>>> the API with a some syntactic sugar like `stream.count().table()` to
>>>>>>>
>>>>>> get
>>>>
>>>>> the overall count of all records (this would obviously not scale,
>>>>>>>
>>>>>> but we
>>>>
>>>>> could support it -- if not now, maybe later).
>>>>>>>
>>>>>>>
>>>>>>> I guess i'd prefer
>>>>>> stream.groupBy().windowBy().count()
>>>>>> stream.groupBy().windowBy().reduce()
>>>>>> stream.groupBy().count()
>>>>>>
>>>>>> As i said above, everything that happens before the final aggregate
>>>>>>
>>>>> call
>>>>
>>>>> can be applied to any of them. So it makes sense to me to do those
>>>>>>
>>>>> things
>>>>
>>>>> ahead of the final aggregate call.
>>>>>>
>>>>>>
>>>>>> Last about builder pattern. I am convinced that we need some
>>>>>>>
>>>>>> "terminal"
>>>>
>>>>> operator/method that tells us when to add the processor to the
>>>>>>>
>>>>>> topology.
>>>>
>>>>> But I don't see the need for a plain builder pattern that feels
>>>>>>>
>>>>>> alien to
>>>>
>>>>> me (see my argument about the second join proposal). Using .stream()
>>>>>>>
>>>>>> /
>>>>
>>>>> .table() as use in many examples might work. But maybe a more generic
>>>>>>> name that we can use in all places like build() or apply() might
>>>>>>>
>>>>>> also be
>>>>
>>>>> an option.
>>>>>>>
>>>>>>>
>>>>>>> Sure, a generic name might be ok.
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>>
>>>>>>>> Thanks Kyle.
>>>>>>>>
>>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>>>>>>>
>>>>>>> winkelman.kyle@gmail.com>
>>>>
>>>>> wrote:
>>>>>>>>
>>>>>>>> Hi Damian,
>>>>>>>>>
>>>>>>>>> When trying to program in the fluent API that has been
>>>>>>>>>>>>>
>>>>>>>>>>>> discussed
>>>>
>>>>> most
>>>>>>>
>>>>>>>> it
>>>>>>>>>
>>>>>>>>>> feels difficult to know when you will actually get an object
>>>>>>>>>>>>>
>>>>>>>>>>>> you
>>>>
>>>>> can
>>>>>
>>>>>> reuse.
>>>>>>>>>
>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>>>>>>>>>
>>>>>>>>>>>> legal
>>>>>
>>>>>> to
>>>>>>>
>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>>>>>>>>>
>>>>>>>>>>>> time?
>>>>>
>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>>>>>>>>>>
>>>>>>>>>> re-use it
>>>>
>>>>> as
>>>>>>>
>>>>>>>> you
>>>>>>>>>
>>>>>>>>>> can today.
>>>>>>>>>>>
>>>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I
>>>>>>>>>
>>>>>>>> see
>>>>
>>>>> it
>>>>>>>
>>>>>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>>>>>>>>>
>>>>>>>> because
>>>>>>>
>>>>>>>> most of this fluent api would continue returning this (this being
>>>>>>>>>
>>>>>>>> the
>>>>
>>>>> builder object currently being manipulated).
>>>>>>>>>
>>>>>>>> So, if you ever store a reference to anything but KStreams and
>>>>>>>>
>>>>>>> KTables
>>>>
>>>>> and
>>>>>>>
>>>>>>>> you use it in two different ways then its possible you make
>>>>>>>>>
>>>>>>>> conflicting
>>>>>
>>>>>> withXXX() calls on the same builder.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> No necessarily true. It could return a new instance of the builder,
>>>>>>>>
>>>>>>> i.e.,
>>>>>
>>>>>> the builders being immutable. So if you held a reference to the
>>>>>>>>
>>>>>>> builder
>>>>
>>>>> it
>>>>>>>
>>>>>>>> would always be the same as it was when it was created.
>>>>>>>>
>>>>>>>>
>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>>>>>>>>
>>>>>>>> kStream.grouped();
>>>>
>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>>
>>>>>>>>> I’ll admit that this shouldn’t happen but some user is going to do
>>>>>>>>>
>>>>>>>> it
>>>>
>>>>> eventually…
>>>>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>>>>>>>>>
>>>>>>>> would
>>>>>
>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>>>>>>>>
>>>>>>>> work
>>>>
>>>>> around would be to always make copies of the config objects you are
>>>>>>>>> building, but this approach has its own problem because now we
>>>>>>>>>
>>>>>>>> have to
>>>>
>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>>>>> processors.
>>>>>>>>>
>>>>>>>>> The point of this long winded example is that we always have to be
>>>>>>>>> thinking about all of the possible ways it could be misused by a
>>>>>>>>>
>>>>>>>> user
>>>>
>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>>
>>>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>>
>>>>>>>>
>>>>>>>> In my attempt at a couple methods with builders I feel that I could
>>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>>
>>>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>>>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("
>>>>> my-store"));
>>>>>
>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>>>>>>>>>
>>>>>>>> Count
>>>>>
>>>>>> for some reason it would throw an error message saying that a store
>>>>>>>>>
>>>>>>>> named
>>>>>>>
>>>>>>>> “my-store” already exists.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>>
>>>>>>>> Cheers,
>>>>>>>> Damian
>>>>>>>>
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>>> Kyle
>>>>>>>>>
>>>>>>>>> From: Damian Guy
>>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>
>>>>>>>>> Hi Kyle,
>>>>>>>>>
>>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>>
>>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>>>>>>>>
>>>>>>>> winkelman.kyle@gmail.com
>>>>
>>>>> wrote:
>>>>>>>>>
>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>>>>>>>>
>>>>>>>>> against
>>>>>
>>>>>> it. The reason I like it is because it makes it clear to the user
>>>>>>>>>>
>>>>>>>>> that
>>>>>
>>>>>> a
>>>>>>>
>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>>>>>>>>
>>>>>>>>> intermediate
>>>>>
>>>>>> class that I need to undetstand.
>>>>>>>>>>
>>>>>>>>>> Yes, that makes sense.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> When trying to program in the fluent API that has been discussed
>>>>>>>>>>
>>>>>>>>> most
>>>>
>>>>> it
>>>>>>>
>>>>>>>> feels difficult to know when you will actually get an object you
>>>>>>>>>>
>>>>>>>>> can
>>>>
>>>>> reuse.
>>>>>>>>>
>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>>>>>>>>
>>>>>>>>> legal
>>>>
>>>>> to
>>>>>
>>>>>> reuse it or does this approach expect you to call grouped each
>>>>>>>>>>
>>>>>>>>> time?
>>>>
>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
>>>>>>>>>
>>>>>>>> it
>>>>
>>>>> as
>>>>>
>>>>>> you
>>>>>>>
>>>>>>>> can today.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>>>>>>>>>
>>>>>>>>> pattern
>>>>
>>>>> I
>>>>>
>>>>>> assume I can reuse everything.
>>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
>>>>>>>>>>
>>>>>>>>> fan
>>>>>
>>>>>> of
>>>>>>>
>>>>>>>> the grouped.
>>>>>>>>>>
>>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>>>>>>>>>
>>>>>>>>> and
>>>>
>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>>>>>>>>
>>>>>>>> want to
>>>>
>>>>> be
>>>>>>>
>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>>>>>>>>>
>>>>>>>> depends
>>>>>>>
>>>>>>>> on maintaining backward compatibility.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>>>>>>>>>
>>>>>>>>> (probably
>>>>
>>>>> 3)
>>>>>
>>>>>> overloads (one for returning a KTable and one for returning a
>>>>>>>>>>
>>>>>>>>> KTable
>>>>
>>>>> with
>>>>>>>
>>>>>>>> Windowed Key, probably would want to split windowed and
>>>>>>>>>>
>>>>>>>>> sessionwindowed
>>>>>
>>>>>> for
>>>>>>>>>
>>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>>>>>>>>
>>>>>>>>> Count,
>>>>>
>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>>>>>>
>>>>>>>>> builder:
>>>>>>>
>>>>>>>> // Count
>>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>>
>>>>>>>>>> groupedStream.count(Count.count().withQueryableStoreName("
>>>>> my-store"));
>>>>>
>>>>>> // Windowed Count
>>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>>
>>>>>>>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>>>>
>>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>
>>>>> // Session Count
>>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>>
>>>>>>>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>>>>
>>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>
>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> // Reduce
>>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>>> Initializer<String> initializer;
>>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>>
>>>>>>>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>>>>
>>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>
>>>>> // Cogroup SessionWindowed
>>>>>>>>>> KTable<String, String> cogrouped =
>>>>>>>>>>
>>>>>>>>> groupedStream1.cogroup(aggregator1)
>>>>>
>>>>>>           .cogroup(groupedStream2, aggregator2)
>>>>>>>>>>           .aggregate(initializer, aggregator,
>>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> public class Count {
>>>>>>>>>>
>>>>>>>>>>       public static class Windowed extends Count {
>>>>>>>>>>           private Windows windows;
>>>>>>>>>>       }
>>>>>>>>>>       public static class SessionWindowed extends Count {
>>>>>>>>>>           private SessionWindows sessionWindows;
>>>>>>>>>>       }
>>>>>>>>>>
>>>>>>>>>>       public static Count count();
>>>>>>>>>>       public static Windowed windowed(Windows windows);
>>>>>>>>>>       public static SessionWindowed sessionWindowed(SessionWindows
>>>>>>>>>> sessionWindows);
>>>>>>>>>>
>>>>>>>>>>       // All withXXX(...) methods.
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>> public class KGroupedStream {
>>>>>>>>>>       public KTable<K, Long> count(Count count);
>>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.Windowed
>>>>>>>>>> count);
>>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>>>>>>>>>>
>>>>>>>>> count);
>>>>>>>
>>>>>>>> …
>>>>>>>>>> }
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Kyle
>>>>>>>>>>
>>>>>>>>>> From: Guozhang Wang
>>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>>
>>>>>>>>>> I played the current proposal a bit with
>>>>>>>>>>
>>>>>>>>> https://github.com/dguy/kafka/
>>>>>>>
>>>>>>>> tree/dsl-experiment <
>>>>>>>>>>
>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>
>>>>>> ,
>>>>>>>>
>>>>>>>>> and here are my observations:
>>>>>>>>>>
>>>>>>>>>> 1. Personally I prefer
>>>>>>>>>>
>>>>>>>>>>       "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>>
>>>>>>>>>> than
>>>>>>>>>>
>>>>>>>>>>       "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>>
>>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it
>>>>>>>>>>
>>>>>>>>> is
>>>>
>>>>> not
>>>>>>>
>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>>>>>>>>>
>>>>>>>>> natural.
>>>>
>>>>> I think it is okay to add the APIs in (
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>>>>
>>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>
>>>>> )
>>>>>>>>>> in KGroupedStream.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>>>>>>>>>>
>>>>>>>>> pass
>>>>>
>>>>>> in
>>>>>>>
>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>>>>>>>>>
>>>>>>>>> return
>>>>
>>>>> RocksDBStore), or it is supposed to return the most-outer supplier
>>>>>>>>>>
>>>>>>>>> with
>>>>>
>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>>>>>>>>
>>>>>>>>> require
>>>>>>>
>>>>>>>> users pass in the inner state store supplier while specifying
>>>>>>>>>>
>>>>>>>>> caching /
>>>>>
>>>>>> logging through other APIs.
>>>>>>>>>>
>>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>>>>>>>>>>
>>>>>>>>> me:
>>>>
>>>>> we
>>>>>
>>>>>> are
>>>>>>>>>
>>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>>>>>>>>>
>>>>>>>>> multiple
>>>>
>>>>> time,
>>>>>>>
>>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why
>>>>>>>>>>
>>>>>>>>> is
>>>>
>>>>> that?
>>>>>>>
>>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>>>>>>>>>>
>>>>>>>>> joins?
>>>>>
>>>>>>
>>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>>>>>>>>>
>>>>>>>>> topology
>>>>
>>>>> so
>>>>>>>
>>>>>>>> I
>>>>>>>>>
>>>>>>>>>> do not see there is a big difference.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>>>>>>>>>>
>>>>>>>>> the
>>>>
>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Below are minor ones:
>>>>>>>>>>
>>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>>
>>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> Guozhang
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>>>>>>
>>>>>>>>> matthias@confluent.io>
>>>>>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>> I see your point about "when to add the processor to the
>>>>>>>>>>>
>>>>>>>>>> topology".
>>>>
>>>>> That
>>>>>>>>>
>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>>>>>>
>>>>>>>>>> topology...
>>>>>>>>>>
>>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>>>>>
>>>>>>>>>> interface
>>>>>>>>>
>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> However, I don't understand your argument about putting
>>>>>>>>>>>
>>>>>>>>>> aggregate()
>>>>
>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>>>>>>>
>>>>>>>>>> parameters
>>>>>>>
>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>>>>>>>>>
>>>>>>>>>> might
>>>>>
>>>>>> be quite confusion for developers.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> -Matthias
>>>>>>>>>>>
>>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>
>>>>>>>>>>>> I also think that mixing optional parameters with configs is a
>>>>>>>>>>>>>
>>>>>>>>>>>> bad
>>>>
>>>>> idea.
>>>>>>>>>>
>>>>>>>>>>> Have not proposal for this atm but just wanted to mention it.
>>>>>>>>>>>>>
>>>>>>>>>>>> Hope
>>>>
>>>>> to
>>>>>>>>>
>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>>>>>>>>>>
>>>>>>>>>>> config
>>>>>
>>>>>> here
>>>>>>>>>>
>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>>>>>>>>>>
>>>>>>>>>>> already
>>>>>
>>>>>> be
>>>>>>>>>
>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>>>>>>>
>>>>>>>>>>>> .groupBy(...)
>>>>>>>>>
>>>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
>>>>>>>>>>>>>
>>>>>>>>>>>> find
>>>>>
>>>>>> some better names).
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>>>>>>>>>>>>
>>>>>>>>>>>> close
>>>>
>>>>> together. I would rather separate both more than less, ie, do
>>>>>>>>>>>>>
>>>>>>>>>>>> into
>>>>
>>>>> the
>>>>>>>>>
>>>>>>>>>> opposite direction.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>>>>>>>>>>>>
>>>>>>>>>>>> The
>>>>
>>>>> initial
>>>>>>>>>>
>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>>
>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>>
>>>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>>>>>>>>>>>
>>>>>>>>>>> KTable
>>>>
>>>>> which
>>>>>>>>>
>>>>>>>>>> is
>>>>>>>>>>>
>>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>>>>>>>>>>
>>>>>>>>>>> "build"
>>>>>
>>>>>> the
>>>>>>>>>>
>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>>>>>>>
>>>>>>>>>>>> optional
>>>>>>>>>
>>>>>>>>>> parameter for count() have to specified on the .grouped() call
>>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>
>>>>> this
>>>>>>>>>
>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>>>>>>>>>>>
>>>>>>>>>>> stream
>>>>
>>>>> that
>>>>>>>>>
>>>>>>>>>> ...
>
> [Message clipped]




-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
It makes me want to cry.

why on earth is the DSL going to expose all its implementation details now?
especially being materialized or not.

If we want to take usefull steps in that direction maybe we are looking 
for a way to let the user switch back and forth between PAPI and DSL?

A change as the proposed would not eliminate any of my pain points while 
still being a heck of work migrating towards to.

Since I am only following this from the point where Eno CC'ed it into 
the users list:

Can someone please rephrase for me what problem this is trying to solve? 
I don't mean to be rude but It uses a problematic feature
"StateStoreSuppliers in DSL" to justify making it even worse. This helps 
us nowhere in making the configs more flexible, its just syntactic sugar.

A low effort shoot like: lets add a properties to operations that would 
otherwise become overloaded to heavy? Or pull the configs by some naming 
schema
form the overall properties. Additionally to that we get rid of 
StateStoreSuppliers in the DSL and have them also configured by said 
properties.

=> way easier to migrate to, way less risk, way more flexible in the 
future (different implementations of the same operation don't require 
code change to configure)

Line 184 makes especially no sense to me. what is a KTableKTable non 
materialized join anyways?

Hope we can discuss more on this.



On 07.07.2017 17:23, Guozhang Wang wrote:
> I messed the indentation on github code repos; this would be easier to read:
>
> https://codeshare.io/GLWW8K
>
>
> Guozhang
>
>
> On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> Hi Damian / Kyle,
>>
>> I think I agree with you guys about the pros / cons of using the builder
>> pattern v.s. using some "secondary classes". And I'm thinking if we can
>> take a "mid" manner between these two. I spent some time with a slight
>> different approach from Damian's current proposal:
>>
>> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
>> java/org/apache/kafka/streams/RefactoredAPIs.java
>>
>> The key idea is to tolerate the final "table()" or "stream()" function to
>> "upgrade" from the secondary classes to the first citizen classes, while
>> having all the specs inside this function. Also this proposal includes some
>> other refactoring that people have been discussed about for the builder to
>> reduce the overloaded functions as well. WDYT?
>>
>>
>> Guozhang
>>
>>
>> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>>
>>> Hi Jan,
>>>
>>> Thanks very much for the input.
>>>
>>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>>> wrote:
>>>
>>>> Hi Damian,
>>>>
>>>> I do see your point of something needs to change. But I fully agree with
>>>> Gouzhang when he says.
>>>> ---
>>>>
>>>> But since this is a incompatibility change, and we are going to remove
>>> the
>>>> compatibility annotations soon it means we only have one chance and we
>>>> really have to make it right.
>>>> ----
>>>>
>>>>
>>> I think we all agree on this one! Hence the discussion.
>>>
>>>
>>>> I fear all suggestions do not go far enough to become something that
>>> will
>>>> carry on for very much longer.
>>>> I am currently working on KAFKA-3705 and try to find the most easy way
>>> for
>>>> the user to give me all the required functionality. The easiest
>>> interface I
>>>> could come up so far can be looked at here.
>>>>
>>>>
>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>> kafka/streams/kstream/internals/KTableImpl.java#L622
>>>>
>>> And its already horribly complicated. I am currently unable to find the
>>>> right abstraction level to have everything falling into place
>>> naturally. To
>>>> be honest I already think introducing
>>>>
>>>>
>>> To be fair that is not a particularly easy problem to solve!
>>>
>>>
>>>> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>>> kafka/streams/kstream/internals/KTableImpl.java#L493
>>>> was unideal and makes everything a mess.
>>>
>>> I'm not sure i agree that it makes everything a mess, but It could have
>>> been done differently.
>>>
>>> The JoinType:Whatever is also not really flexible. 2 things come to my
>>> mind:
>>>> 1. I don't think we should rule out config based decisions say configs
>>> like
>>>>          streams.$applicationID.joins.$joinname.conf = value
>>>>
>>> Is this just for config? Or are you suggesting that we could somehow
>>> "code"
>>> the join in a config file?
>>>
>>>
>>>> This can allow for tremendous changes without single API change and IMO
>>> it
>>>> was not considered enough yet.
>>>>
>>>> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>>>> example can be used to implement different join types as the user
>>> wishes.
>>> Do you have an example of how this might look?
>>>
>>>
>>>> As Gouzhang said: stopping to break users is very important.
>>>
>>> Of course. We want to make it as easy as possible for people to use
>>> streams.
>>>
>>>
>>> especially with this changes + All the plans I sadly only have in my head
>>>> but hopefully the first link can give a glimpse.
>>>>
>>>> Thanks for preparing the examples made it way clearer to me what exactly
>>>> we are talking about. I would argue to go a bit slower and more
>>> carefull on
>>>> this one. At some point we need to get it right. Peeking over to the
>>> hadoop
>>>> guys with their hughe userbase. Config files really work well for them.
>>>>
>>>> Best Jan
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> On 30.06.2017 09:31, Damian Guy wrote:
>>>>> Thanks Matthias
>>>>>
>>>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>>>> wrote:
>>>>>> I am just catching up on this thread, so sorry for the long email in
>>>>>> advance... Also, it's to some extend a dump of thoughts and not
>>> always a
>>>>>> clear proposal. Still need to think about this in more detail. But
>>> maybe
>>>>>> it helps other to get new ideas :)
>>>>>>
>>>>>>
>>>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>> parameters
>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>> might
>>>>>>>> be quite confusion for developers.
>>>>>>>>
>>>>>>>>
>>>>>>> I see what you are saying, but the grouped stream is effectively a
>>>> no-op
>>>>>>> until you call one of the aggregate/count/reduce etc functions. So
>>> the
>>>>>>> optional params are ones that are applicable to any of the
>>> operations
>>>> you
>>>>>>> can perform on this grouped stream. Then the final
>>>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>>>> required/specific to that function.
>>>>>>>
>>>>>> I understand your argument, but you don't share the conclusion. If we
>>>>>> need a "final/terminal" call, the better way might be
>>>>>>
>>>>>> .groupBy().count().withXX().build()
>>>>>>
>>>>>> (with a better name for build() though)
>>>>>>
>>>>>>
>>>>> The point is that all the other calls, i.e,withBlah, windowed, etc
>>> apply
>>>>> too all the aggregate functions. The terminal call being the actual
>>> type
>>>> of
>>>>> aggregation you want to do. I personally find this more natural than
>>>>> groupBy().count().withBlah().build()
>>>>>
>>>>>
>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>> I like this. However, I don't see a reason to have windowed() and
>>>>>> sessionWindowed(). We should have one top-level `Windows` interface
>>> that
>>>>>> both `TimeWindows` and `SessionWindows` implement and just have a
>>> single
>>>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>>>> separation of `SessionWindows` in the first place, and this seems to
>>> be
>>>>>> an opportunity to clean this up. It was hard to change when we
>>>>>> introduced session windows)
>>>>>>
>>>>> Yes - true we should look into that.
>>>>>
>>>>>
>>>>>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>>>>>> might also want to use windowBy() (instead of windowed()). Not sure
>>> how
>>>>>> important this is, but it seems to be inconsistent otherwise.
>>>>>>
>>>>>>
>>>>> Makes sense
>>>>>
>>>>>
>>>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>>> think,
>>>>>> defining an inner/left/outer join is not an optional argument but a
>>>>>> first class concept and should have a proper representation in the
>>> API
>>>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>>>
>>>>>>
>>>>> Yep, i did originally have it as a required param and maybe that is
>>> what
>>>> we
>>>>> go with. It could have a default, but maybe that is confusing.
>>>>>
>>>>>
>>>>>
>>>>>> About the two join API proposals, the second one has too much boiler
>>>>>> plate code for my taste. Also, the actual join() operator has only
>>> one
>>>>>> argument what is weird to me, as in my thinking process, the main
>>>>>> operator call, should have one parameter per mandatory argument but
>>> your
>>>>>> proposal put the mandatory arguments into Joins.streamStreamJoin()
>>> call.
>>>>>> This is far from intuitive IMHO.
>>>>>>
>>>>>>
>>>>> This is the builder pattern, you only need one param as the builder
>>> has
>>>>> captured all of the required and optional arguments.
>>>>>
>>>>>
>>>>>> The first join proposal also seems to align better with the pattern
>>>>>> suggested for aggregations and having the same pattern for all
>>> operators
>>>>>> is important (as you stated already).
>>>>>>
>>>>>>
>>>>> This is why i offered two alternatives as i started out with. 1 is the
>>>>> builder pattern, the other is the more fluent pattern.
>>>>>
>>>>>
>>>>>> Coming back to the config vs optional parameter. What about having a
>>>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>>>
>>>>>>
>>>>> Sure, it is currently called withLogConfig() as that is the only thing
>>>> that
>>>>> is really config.
>>>>>
>>>>>
>>>>>> This also raises the question if until() is a windows property?
>>>>>> Actually, until() seems to be a configuration parameter and thus,
>>> should
>>>>>> not not have it's own method.
>>>>>>
>>>>>>
>>>>> Hmmm, i don't agree. Until is a property of the window. It is going
>>> to be
>>>>> potentially different for every window operation you do in a streams
>>> app.
>>>>>
>>>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>>>
>>>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>>>    groupedStream.counting()
>>>>>>>                    .windowed(TimeWindows.of(10L).until(10))
>>>>>>>                    .table();
>>>>>> This is an interesting idea, and it remind my on some feedback about
>>> "I
>>>>>> wanted to count a stream, but there was no count() method -- I first
>>>>>> needed to figure out, that I need to group the stream first to be
>>> able
>>>>>> to count it. It does make sense in hindsight but was not obvious in
>>> the
>>>>>> beginning". Thus, carrying out this thought, we could also do the
>>>>>> following:
>>>>>>
>>>>>> stream.count().groupedBy().windowedBy().table();
>>>>>>
>>>>>> -> Note, I use "grouped" and "windowed" instead of imperative here,
>>> as
>>>>>> it comes after the count()
>>>>>>
>>>>>> This would be more consistent than your proposal (that has grouping
>>>>>> before but windowing after count()). It might even allow us to enrich
>>>>>> the API with a some syntactic sugar like `stream.count().table()` to
>>> get
>>>>>> the overall count of all records (this would obviously not scale,
>>> but we
>>>>>> could support it -- if not now, maybe later).
>>>>>>
>>>>>>
>>>>> I guess i'd prefer
>>>>> stream.groupBy().windowBy().count()
>>>>> stream.groupBy().windowBy().reduce()
>>>>> stream.groupBy().count()
>>>>>
>>>>> As i said above, everything that happens before the final aggregate
>>> call
>>>>> can be applied to any of them. So it makes sense to me to do those
>>> things
>>>>> ahead of the final aggregate call.
>>>>>
>>>>>
>>>>>> Last about builder pattern. I am convinced that we need some
>>> "terminal"
>>>>>> operator/method that tells us when to add the processor to the
>>> topology.
>>>>>> But I don't see the need for a plain builder pattern that feels
>>> alien to
>>>>>> me (see my argument about the second join proposal). Using .stream()
>>> /
>>>>>> .table() as use in many examples might work. But maybe a more generic
>>>>>> name that we can use in all places like build() or apply() might
>>> also be
>>>>>> an option.
>>>>>>
>>>>>>
>>>>> Sure, a generic name might be ok.
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>>
>>>>>>
>>>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>>>> Thanks Kyle.
>>>>>>>
>>>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>>> winkelman.kyle@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Damian,
>>>>>>>>
>>>>>>>>>>>> When trying to program in the fluent API that has been
>>> discussed
>>>>>> most
>>>>>>>> it
>>>>>>>>>>>> feels difficult to know when you will actually get an object
>>> you
>>>> can
>>>>>>>> reuse.
>>>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>>> legal
>>>>>> to
>>>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>>> time?
>>>>>>>>>> I'd anticipate that once you have a KGroupedStream you can
>>> re-use it
>>>>>> as
>>>>>>>> you
>>>>>>>>>> can today.
>>>>>>>> You said it yourself in another post that the grouped stream is
>>>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I
>>> see
>>>>>> it
>>>>>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>>>>>> because
>>>>>>>> most of this fluent api would continue returning this (this being
>>> the
>>>>>>>> builder object currently being manipulated).
>>>>>>> So, if you ever store a reference to anything but KStreams and
>>> KTables
>>>>>> and
>>>>>>>> you use it in two different ways then its possible you make
>>>> conflicting
>>>>>>>> withXXX() calls on the same builder.
>>>>>>>>
>>>>>>>>
>>>>>>> No necessarily true. It could return a new instance of the builder,
>>>> i.e.,
>>>>>>> the builders being immutable. So if you held a reference to the
>>> builder
>>>>>> it
>>>>>>> would always be the same as it was when it was created.
>>>>>>>
>>>>>>>
>>>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>>> kStream.grouped();
>>>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>>>
>>>>>>>> I’ll admit that this shouldn’t happen but some user is going to do
>>> it
>>>>>>>> eventually…
>>>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>>>> would
>>>>>>>> most likely be equivalent to the version withDeclaredSerdes. One
>>> work
>>>>>>>> around would be to always make copies of the config objects you are
>>>>>>>> building, but this approach has its own problem because now we
>>> have to
>>>>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>>>> processors.
>>>>>>>>
>>>>>>>> The point of this long winded example is that we always have to be
>>>>>>>> thinking about all of the possible ways it could be misused by a
>>> user
>>>>>>>> (causing them to see hard to diagnose problems).
>>>>>>>>
>>>>>>> Exactly! That is the point of the discussion really.
>>>>>>>
>>>>>>>
>>>>>>>> In my attempt at a couple methods with builders I feel that I could
>>>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>>>> // Count
>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>
>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>>>> Count
>>>>>>>> for some reason it would throw an error message saying that a store
>>>>>> named
>>>>>>>> “my-store” already exists.
>>>>>>>>
>>>>>>>>
>>>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Damian
>>>>>>>
>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Kyle
>>>>>>>>
>>>>>>>> From: Damian Guy
>>>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>>>> To: dev@kafka.apache.org
>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>
>>>>>>>> Hi Kyle,
>>>>>>>>
>>>>>>>> Thanks for your input. Really appreciated.
>>>>>>>>
>>>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>>> winkelman.kyle@gmail.com
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> I like more of a builder pattern even though others have voiced
>>>> against
>>>>>>>>> it. The reason I like it is because it makes it clear to the user
>>>> that
>>>>>> a
>>>>>>>>> call to KGroupedStream#count will return a KTable not some
>>>> intermediate
>>>>>>>>> class that I need to undetstand.
>>>>>>>>>
>>>>>>>> Yes, that makes sense.
>>>>>>>>
>>>>>>>>
>>>>>>>>> When trying to program in the fluent API that has been discussed
>>> most
>>>>>> it
>>>>>>>>> feels difficult to know when you will actually get an object you
>>> can
>>>>>>>> reuse.
>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>>> legal
>>>> to
>>>>>>>>> reuse it or does this approach expect you to call grouped each
>>> time?
>>>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
>>> it
>>>> as
>>>>>> you
>>>>>>>> can today.
>>>>>>>>
>>>>>>>>
>>>>>>>>> This question doesn’t pop into my head at all in the builder
>>> pattern
>>>> I
>>>>>>>>> assume I can reuse everything.
>>>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
>>>> fan
>>>>>> of
>>>>>>>>> the grouped.
>>>>>>>>>
>>>>>>>>> Yes, grouped() was more for demonstration and because groupBy()
>>> and
>>>>>>>> groupByKey() were taken! So i'd imagine the api would actually
>>> want to
>>>>>> be
>>>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>>>>>> depends
>>>>>>>> on maintaining backward compatibility.
>>>>>>>>
>>>>>>>>
>>>>>>>>> Unfortunately, the below approach would require atleast 2
>>> (probably
>>>> 3)
>>>>>>>>> overloads (one for returning a KTable and one for returning a
>>> KTable
>>>>>> with
>>>>>>>>> Windowed Key, probably would want to split windowed and
>>>> sessionwindowed
>>>>>>>> for
>>>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>>>> Count,
>>>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>>>> builder:
>>>>>>>>> // Count
>>>>>>>>> KTable<String, Long> count =
>>>>>>>>>
>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>>> // Windowed Count
>>>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>>>
>>>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>>> (10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>>> // Session Count
>>>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>>>
>>>> groupedStream.count(Count.sessionWindowed(SessionWindows.
>>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>> Above and below, i think i'd prefer it to be:
>>>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>> // Reduce
>>>>>>>>> Reducer<Long> reducer;
>>>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>>>
>>>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>>>> Initializer<String> initializer;
>>>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>>>
>>>> Aggregate.windowed(TimeWindows.of(10L).until(10)).
>>> withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>>> // Cogroup SessionWindowed
>>>>>>>>> KTable<String, String> cogrouped =
>>>> groupedStream1.cogroup(aggregator1)
>>>>>>>>>           .cogroup(groupedStream2, aggregator2)
>>>>>>>>>           .aggregate(initializer, aggregator,
>>>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> public class Count {
>>>>>>>>>
>>>>>>>>>       public static class Windowed extends Count {
>>>>>>>>>           private Windows windows;
>>>>>>>>>       }
>>>>>>>>>       public static class SessionWindowed extends Count {
>>>>>>>>>           private SessionWindows sessionWindows;
>>>>>>>>>       }
>>>>>>>>>
>>>>>>>>>       public static Count count();
>>>>>>>>>       public static Windowed windowed(Windows windows);
>>>>>>>>>       public static SessionWindowed sessionWindowed(SessionWindows
>>>>>>>>> sessionWindows);
>>>>>>>>>
>>>>>>>>>       // All withXXX(...) methods.
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>> public class KGroupedStream {
>>>>>>>>>       public KTable<K, Long> count(Count count);
>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>>>>>>>       public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>>>>>> count);
>>>>>>>>> …
>>>>>>>>> }
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Kyle
>>>>>>>>>
>>>>>>>>> From: Guozhang Wang
>>>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>>>> To: dev@kafka.apache.org
>>>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>>>
>>>>>>>>> I played the current proposal a bit with
>>>>>> https://github.com/dguy/kafka/
>>>>>>>>> tree/dsl-experiment <
>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>> ,
>>>>>>>>> and here are my observations:
>>>>>>>>>
>>>>>>>>> 1. Personally I prefer
>>>>>>>>>
>>>>>>>>>       "stream.group(mapper) / stream.groupByKey()"
>>>>>>>>>
>>>>>>>>> than
>>>>>>>>>
>>>>>>>>>       "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>>>
>>>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it
>>> is
>>>>>> not
>>>>>>>>> "really" optional like others, 2) syntax-wise it reads more
>>> natural.
>>>>>>>>> I think it is okay to add the APIs in (
>>>>>>>>>
>>>>>>>>>
>>>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>>>> )
>>>>>>>>> in KGroupedStream.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>>>> pass
>>>>>> in
>>>>>>>>> the most-inner state store supplier (e.g. then one whose get()
>>> return
>>>>>>>>> RocksDBStore), or it is supposed to return the most-outer supplier
>>>> with
>>>>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>>>> require
>>>>>>>>> users pass in the inner state store supplier while specifying
>>>> caching /
>>>>>>>>> logging through other APIs.
>>>>>>>>>
>>>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>>> me:
>>>> we
>>>>>>>> are
>>>>>>>>> allowing users to call other APIs like "withQueryableName"
>>> multiple
>>>>>> time,
>>>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why
>>> is
>>>>>> that?
>>>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>>>> joins?
>>>>>>>>>
>>>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>>>> StateStoreSupplier it will still be user code specifying the
>>> topology
>>>>>> so
>>>>>>>> I
>>>>>>>>> do not see there is a big difference.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>>> the
>>>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Below are minor ones:
>>>>>>>>>
>>>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>>>
>>>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>>>> matthias@confluent.io>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> I see your point about "when to add the processor to the
>>> topology".
>>>>>>>> That
>>>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>>>> topology...
>>>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>>>> interface
>>>>>>>>>> -- but this might be subjective.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> However, I don't understand your argument about putting
>>> aggregate()
>>>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>>>> parameters
>>>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>>>> might
>>>>>>>>>> be quite confusion for developers.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>>>> I also think that mixing optional parameters with configs is a
>>> bad
>>>>>>>>> idea.
>>>>>>>>>>>> Have not proposal for this atm but just wanted to mention it.
>>> Hope
>>>>>>>> to
>>>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>>>> config
>>>>>>>>> here
>>>>>>>>>>> is the logging config - which we don't really need as it can
>>>> already
>>>>>>>> be
>>>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>>>> .groupBy(...)
>>>>>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
>>>> find
>>>>>>>>>>>> some better names).
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too
>>> close
>>>>>>>>>>>> together. I would rather separate both more than less, ie, do
>>> into
>>>>>>>> the
>>>>>>>>>>>> opposite direction.
>>>>>>>>>>>>
>>>>>>>>>>>> I am also wondering, if we could so something more "fluent".
>>> The
>>>>>>>>> initial
>>>>>>>>>>>> proposal was like:
>>>>>>>>>>>>
>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>>>
>>>>>>>>>>> I agree, but then all of the withXXX methods need to be on
>>> KTable
>>>>>>>> which
>>>>>>>>>> is
>>>>>>>>>>> worse in my opinion. You also need something that is going to
>>>> "build"
>>>>>>>>> the
>>>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>>>> optional
>>>>>>>>>>>> parameter for count() have to specified on the .grouped() call
>>> --
>>>>>>>> this
>>>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> I actually prefer this method as you are building a grouped
>>> stream
>>>>>>>> that
>>>>>>>>>> you
>>>>>>>>>>> will aggregate. So
>>>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>>>> etc
>>>>>>>>>>> seems natural to me.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> I did not think this through in detail, but can't we just do
>>> the
>>>>>>>>> initial
>>>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>>>
>>>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>>>
>>>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>>>> .withXXX()
>>>>>>>>> are
>>>>>>>>>>>> just added to the KTable interface. Or do I miss anything why
>>> this
>>>>>>>>> wont'
>>>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>> See above.
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>
>>>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>>>>>> fluent
>>>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>>> in
>>>>>>>> the
>>>>>>>>>> DSL,
>>>>>>>>>>>>> but i think that enabling caching and enabling logging are
>>> things
>>>>>>>>> that
>>>>>>>>>>>>> aren't actually config. I'd probably not add
>>> withLogConfig(...)
>>>>>>>> (even
>>>>>>>>>>>>> though it is below) as this is actually config and we already
>>>> have
>>>>>>>> a
>>>>>>>>>> way
>>>>>>>>>>>> of
>>>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>>> the
>>>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
>>>> that
>>>>>>>>> is
>>>>>>>>>> a
>>>>>>>>>>>>> bit of a tedious process for someone that just wants to use
>>> the
>>>>>>>>> default
>>>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>>>
>>>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>>> to....
>>>> If
>>>>>>>>> you
>>>>>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>>>>>> logged
>>>>>>>>>> etc,
>>>>>>>>>>>>> you would currently need to implement that yourself. Ideally
>>> we
>>>> can
>>>>>>>>>>>> provide
>>>>>>>>>>>>> a way where we will wrap the custom store with logging,
>>> metrics,
>>>>>>>>> etc. I
>>>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>>>> appropriate
>>>>>>>>> on
>>>>>>>>>>>> the
>>>>>>>>>>>>> Stores API.
>>>>>>>>>>>>>
>>>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>>>>>> // count with mapped key
>>>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>>>           .withKeyMapper(keyMapper)
>>>>>>>>>>>>>           .withKeySerde(Serdes.Long())
>>>>>>>>>>>>>           .withValueSerde(Serdes.String())
>>>>>>>>>>>>>           .withQueryableName("my-store")
>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>
>>>>>>>>>>>>> // windowed count
>>>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>>>> stream.grouped()
>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>
>>>>>>>>>>>>> // windowed reduce
>>>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>           .withQueryableName("my-window-store")
>>>>>>>>>>>>>           .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>>>           .reduce(windowedReducer);
>>>>>>>>>>>>>
>>>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>>>
>>>>>>>>>>>>> // aggregate
>>>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>>>           .withQueryableName("my-aggregate-store")
>>>>>>>>>>>>>           .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>>>
>>>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>>>> stateStoreSupplier
>>>>>>>>>>>> = null;
>>>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>>> stream.grouped()
>>>>>>>>>>>>>           .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>>>           .aggregate(aggregator, init);
>>>>>>>>>>>>>
>>>>>>>>>>>>> // disable caching
>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>           .withQueryableName("name")
>>>>>>>>>>>>>           .withCachingEnabled(false)
>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>
>>>>>>>>>>>>> // disable logging
>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>           .withQueryableName("q")
>>>>>>>>>>>>>           .withLoggingEnabled(false)
>>>>>>>>>>>>>           .count();
>>>>>>>>>>>>>
>>>>>>>>>>>>> // override log config
>>>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>>>           .withLogConfig(Collections.sin
>>> gletonMap("segment.size",
>>>>>>>>> "10"))
>>>>>>>>>>>>>           .reduce(reducer);
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> If anyone wants to play around with this you can find the code
>>>>>>>> here:
>>>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>>>
>>>>>>>>>>>>> Note: It won't actually work as most of the methods just
>>> return
>>>>>>>> null.
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>>>>>> wrote:
>>>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>>> both
>>>>>>>> are
>>>>>>>>>>>> better
>>>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> The fluent API approach reads better, no mention of builder
>>> or
>>>>>>>> build
>>>>>>>>>>>>>> anywhere. The main downside is that the method signatures
>>> are a
>>>>>>>>> little
>>>>>>>>>>>> less
>>>>>>>>>>>>>> clear. By reading the method signature, one doesn't
>>> necessarily
>>>>>>>>> knows
>>>>>>>>>>>> what
>>>>>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>>>>>> (`table()`
>>>>>>>>>>>> in
>>>>>>>>>>>>>> this case) that gives you what you actually care about
>>> (`KTable`
>>>>>>>> in
>>>>>>>>>> this
>>>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>>>>>>>> comparison.
>>>>>>>>>>>>>> The builder approach avoids the issues mentioned above, but
>>> it
>>>>>>>>> doesn't
>>>>>>>>>>>> read
>>>>>>>>>>>>>> as well.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> Ismael
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>>>> damian.guy@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>>>> choices
>>>>>>>>>> we've
>>>>>>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>>>>>>>> operations
>>>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>>> API,
>>>>>>>> i.e,
>>>>>>>>>>>> there
>>>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>>>>>> noisy
>>>>>>>>>> and
>>>>>>>>>>>> i
>>>>>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>>>>>> params.
>>>>>>>>> In
>>>>>>>>>>>>>>> particular we've had some requests to be able to turn
>>> caching
>>>>>>>> off,
>>>>>>>>> or
>>>>>>>>>>>>>>> change log configs,  on a per operator basis (note this can
>>> be
>>>>>>>> done
>>>>>>>>>> now
>>>>>>>>>>>>>> if
>>>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>>>> cumbersome).
>>>>>>>>>>>>>>> So this is a bit of an open question. How can we change the
>>> DSL
>>>>>>>>>>>> overloads
>>>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>>>> easily
>>>>>>>>>>>> extended
>>>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> One option would be to use a fluent API approach for
>>> providing
>>>>>>>> the
>>>>>>>>>>>>>> optional
>>>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>>>      .withStoreName("name")
>>>>>>>>>>>>>>>      .withCachingEnabled(false)
>>>>>>>>>>>>>>>      .withLoggingEnabled(config)
>>>>>>>>>>>>>>>      .table()
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>>>> method,
>>>>>>>>> so
>>>>>>>>>> it
>>>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>>>> CountBuilder("storeName").with
>>> CachingEnabled(false).build())
>>>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>>> you
>>>> on
>>>>>>>>>>>> about!
>>>>>>>>>>>>>>> The above has focussed on state store related overloads, but
>>>> the
>>>>>>>>> same
>>>>>>>>>>>>>> ideas
>>>>>>>>>>>>>>> could  be applied to joins etc, where we presently have many
>>>> join
>>>>>>>>>>>> methods
>>>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> -- Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>
>>
>>
>> --
>> -- Guozhang
>>
>
>



Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
I messed the indentation on github code repos; this would be easier to read:

https://codeshare.io/GLWW8K


Guozhang


On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi Damian / Kyle,
>
> I think I agree with you guys about the pros / cons of using the builder
> pattern v.s. using some "secondary classes". And I'm thinking if we can
> take a "mid" manner between these two. I spent some time with a slight
> different approach from Damian's current proposal:
>
> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
> java/org/apache/kafka/streams/RefactoredAPIs.java
>
> The key idea is to tolerate the final "table()" or "stream()" function to
> "upgrade" from the secondary classes to the first citizen classes, while
> having all the specs inside this function. Also this proposal includes some
> other refactoring that people have been discussed about for the builder to
> reduce the overloaded functions as well. WDYT?
>
>
> Guozhang
>
>
> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>
>> Hi Jan,
>>
>> Thanks very much for the input.
>>
>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>> > Hi Damian,
>> >
>> > I do see your point of something needs to change. But I fully agree with
>> > Gouzhang when he says.
>> > ---
>> >
>> > But since this is a incompatibility change, and we are going to remove
>> the
>> > compatibility annotations soon it means we only have one chance and we
>> > really have to make it right.
>> > ----
>> >
>> >
>> I think we all agree on this one! Hence the discussion.
>>
>>
>> > I fear all suggestions do not go far enough to become something that
>> will
>> > carry on for very much longer.
>> > I am currently working on KAFKA-3705 and try to find the most easy way
>> for
>> > the user to give me all the required functionality. The easiest
>> interface I
>> > could come up so far can be looked at here.
>> >
>> >
>> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> kafka/streams/kstream/internals/KTableImpl.java#L622
>> >
>> >
>> And its already horribly complicated. I am currently unable to find the
>> > right abstraction level to have everything falling into place
>> naturally. To
>> > be honest I already think introducing
>> >
>> >
>> To be fair that is not a particularly easy problem to solve!
>>
>>
>> >
>> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> kafka/streams/kstream/internals/KTableImpl.java#L493
>> >
>> > was unideal and makes everything a mess.
>>
>>
>> I'm not sure i agree that it makes everything a mess, but It could have
>> been done differently.
>>
>> The JoinType:Whatever is also not really flexible. 2 things come to my
>> mind:
>> >
>> > 1. I don't think we should rule out config based decisions say configs
>> like
>> >         streams.$applicationID.joins.$joinname.conf = value
>> >
>>
>> Is this just for config? Or are you suggesting that we could somehow
>> "code"
>> the join in a config file?
>>
>>
>> > This can allow for tremendous changes without single API change and IMO
>> it
>> > was not considered enough yet.
>> >
>> > 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>> > example can be used to implement different join types as the user
>> wishes.
>> >
>>
>> Do you have an example of how this might look?
>>
>>
>> > As Gouzhang said: stopping to break users is very important.
>>
>>
>> Of course. We want to make it as easy as possible for people to use
>> streams.
>>
>>
>> especially with this changes + All the plans I sadly only have in my head
>> > but hopefully the first link can give a glimpse.
>> >
>> > Thanks for preparing the examples made it way clearer to me what exactly
>> > we are talking about. I would argue to go a bit slower and more
>> carefull on
>> > this one. At some point we need to get it right. Peeking over to the
>> hadoop
>> > guys with their hughe userbase. Config files really work well for them.
>> >
>> > Best Jan
>> >
>> >
>> >
>> >
>> >
>> > On 30.06.2017 09:31, Damian Guy wrote:
>> > > Thanks Matthias
>> > >
>> > > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> > >
>> > >> I am just catching up on this thread, so sorry for the long email in
>> > >> advance... Also, it's to some extend a dump of thoughts and not
>> always a
>> > >> clear proposal. Still need to think about this in more detail. But
>> maybe
>> > >> it helps other to get new ideas :)
>> > >>
>> > >>
>> > >>>> However, I don't understand your argument about putting aggregate()
>> > >>>> after the withXX() -- all the calls to withXX() set optional
>> > parameters
>> > >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> > >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>> might
>> > >>>> be quite confusion for developers.
>> > >>>>
>> > >>>>
>> > >>> I see what you are saying, but the grouped stream is effectively a
>> > no-op
>> > >>> until you call one of the aggregate/count/reduce etc functions. So
>> the
>> > >>> optional params are ones that are applicable to any of the
>> operations
>> > you
>> > >>> can perform on this grouped stream. Then the final
>> > >>> count()/reduce()/aggregate() call has any of the params that are
>> > >>> required/specific to that function.
>> > >>>
>> > >> I understand your argument, but you don't share the conclusion. If we
>> > >> need a "final/terminal" call, the better way might be
>> > >>
>> > >> .groupBy().count().withXX().build()
>> > >>
>> > >> (with a better name for build() though)
>> > >>
>> > >>
>> > > The point is that all the other calls, i.e,withBlah, windowed, etc
>> apply
>> > > too all the aggregate functions. The terminal call being the actual
>> type
>> > of
>> > > aggregation you want to do. I personally find this more natural than
>> > > groupBy().count().withBlah().build()
>> > >
>> > >
>> > >>> groupedStream.count(/** non windowed count**/)
>> > >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>> > >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>> > >>
>> > >> I like this. However, I don't see a reason to have windowed() and
>> > >> sessionWindowed(). We should have one top-level `Windows` interface
>> that
>> > >> both `TimeWindows` and `SessionWindows` implement and just have a
>> single
>> > >> windowed() method that accepts all `Windows`. (I did not like the
>> > >> separation of `SessionWindows` in the first place, and this seems to
>> be
>> > >> an opportunity to clean this up. It was hard to change when we
>> > >> introduced session windows)
>> > >>
>> > > Yes - true we should look into that.
>> > >
>> > >
>> > >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>> > >> might also want to use windowBy() (instead of windowed()). Not sure
>> how
>> > >> important this is, but it seems to be inconsistent otherwise.
>> > >>
>> > >>
>> > > Makes sense
>> > >
>> > >
>> > >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>> think,
>> > >> defining an inner/left/outer join is not an optional argument but a
>> > >> first class concept and should have a proper representation in the
>> API
>> > >> (like the current methods join(), leftJoin, outerJoin()).
>> > >>
>> > >>
>> > > Yep, i did originally have it as a required param and maybe that is
>> what
>> > we
>> > > go with. It could have a default, but maybe that is confusing.
>> > >
>> > >
>> > >
>> > >> About the two join API proposals, the second one has too much boiler
>> > >> plate code for my taste. Also, the actual join() operator has only
>> one
>> > >> argument what is weird to me, as in my thinking process, the main
>> > >> operator call, should have one parameter per mandatory argument but
>> your
>> > >> proposal put the mandatory arguments into Joins.streamStreamJoin()
>> call.
>> > >> This is far from intuitive IMHO.
>> > >>
>> > >>
>> > > This is the builder pattern, you only need one param as the builder
>> has
>> > > captured all of the required and optional arguments.
>> > >
>> > >
>> > >> The first join proposal also seems to align better with the pattern
>> > >> suggested for aggregations and having the same pattern for all
>> operators
>> > >> is important (as you stated already).
>> > >>
>> > >>
>> > > This is why i offered two alternatives as i started out with. 1 is the
>> > > builder pattern, the other is the more fluent pattern.
>> > >
>> > >
>> > >>
>> > >> Coming back to the config vs optional parameter. What about having a
>> > >> method withConfig[s](...) that allow to put in the configuration?
>> > >>
>> > >>
>> > > Sure, it is currently called withLogConfig() as that is the only thing
>> > that
>> > > is really config.
>> > >
>> > >
>> > >> This also raises the question if until() is a windows property?
>> > >> Actually, until() seems to be a configuration parameter and thus,
>> should
>> > >> not not have it's own method.
>> > >>
>> > >>
>> > > Hmmm, i don't agree. Until is a property of the window. It is going
>> to be
>> > > potentially different for every window operation you do in a streams
>> app.
>> > >
>> > >
>> > >>
>> > >> Browsing throw your example DSL branch, I also saw this one:
>> > >>
>> > >>> final KTable<Windowed<String>, Long> windowed>
>> > >>   groupedStream.counting()
>> > >>>                   .windowed(TimeWindows.of(10L).until(10))
>> > >>>                   .table();
>> > >> This is an interesting idea, and it remind my on some feedback about
>> "I
>> > >> wanted to count a stream, but there was no count() method -- I first
>> > >> needed to figure out, that I need to group the stream first to be
>> able
>> > >> to count it. It does make sense in hindsight but was not obvious in
>> the
>> > >> beginning". Thus, carrying out this thought, we could also do the
>> > >> following:
>> > >>
>> > >> stream.count().groupedBy().windowedBy().table();
>> > >>
>> > >> -> Note, I use "grouped" and "windowed" instead of imperative here,
>> as
>> > >> it comes after the count()
>> > >>
>> > >> This would be more consistent than your proposal (that has grouping
>> > >> before but windowing after count()). It might even allow us to enrich
>> > >> the API with a some syntactic sugar like `stream.count().table()` to
>> get
>> > >> the overall count of all records (this would obviously not scale,
>> but we
>> > >> could support it -- if not now, maybe later).
>> > >>
>> > >>
>> > > I guess i'd prefer
>> > > stream.groupBy().windowBy().count()
>> > > stream.groupBy().windowBy().reduce()
>> > > stream.groupBy().count()
>> > >
>> > > As i said above, everything that happens before the final aggregate
>> call
>> > > can be applied to any of them. So it makes sense to me to do those
>> things
>> > > ahead of the final aggregate call.
>> > >
>> > >
>> > >> Last about builder pattern. I am convinced that we need some
>> "terminal"
>> > >> operator/method that tells us when to add the processor to the
>> topology.
>> > >> But I don't see the need for a plain builder pattern that feels
>> alien to
>> > >> me (see my argument about the second join proposal). Using .stream()
>> /
>> > >> .table() as use in many examples might work. But maybe a more generic
>> > >> name that we can use in all places like build() or apply() might
>> also be
>> > >> an option.
>> > >>
>> > >>
>> > > Sure, a generic name might be ok.
>> > >
>> > >
>> > >
>> > >
>> > >> -Matthias
>> > >>
>> > >>
>> > >>
>> > >> On 6/29/17 7:37 AM, Damian Guy wrote:
>> > >>> Thanks Kyle.
>> > >>>
>> > >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>> winkelman.kyle@gmail.com>
>> > >>> wrote:
>> > >>>
>> > >>>> Hi Damian,
>> > >>>>
>> > >>>>>>>> When trying to program in the fluent API that has been
>> discussed
>> > >> most
>> > >>>> it
>> > >>>>>>>> feels difficult to know when you will actually get an object
>> you
>> > can
>> > >>>> reuse.
>> > >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>> > legal
>> > >> to
>> > >>>>>>>> reuse it or does this approach expect you to call grouped each
>> > time?
>> > >>>>>> I'd anticipate that once you have a KGroupedStream you can
>> re-use it
>> > >> as
>> > >>>> you
>> > >>>>>> can today.
>> > >>>> You said it yourself in another post that the grouped stream is
>> > >>>> effectively a no-op until a count, reduce, or aggregate. The way I
>> see
>> > >> it
>> > >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>> > >> because
>> > >>>> most of this fluent api would continue returning this (this being
>> the
>> > >>>> builder object currently being manipulated).
>> > >>> So, if you ever store a reference to anything but KStreams and
>> KTables
>> > >> and
>> > >>>> you use it in two different ways then its possible you make
>> > conflicting
>> > >>>> withXXX() calls on the same builder.
>> > >>>>
>> > >>>>
>> > >>> No necessarily true. It could return a new instance of the builder,
>> > i.e.,
>> > >>> the builders being immutable. So if you held a reference to the
>> builder
>> > >> it
>> > >>> would always be the same as it was when it was created.
>> > >>>
>> > >>>
>> > >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>> kStream.grouped();
>> > >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>> > >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>> > >>>>
>> > >>>> I’ll admit that this shouldn’t happen but some user is going to do
>> it
>> > >>>> eventually…
>> > >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>> > would
>> > >>>> most likely be equivalent to the version withDeclaredSerdes. One
>> work
>> > >>>> around would be to always make copies of the config objects you are
>> > >>>> building, but this approach has its own problem because now we
>> have to
>> > >>>> identify which configs are equivalent so we don’t create repeated
>> > >>>> processors.
>> > >>>>
>> > >>>> The point of this long winded example is that we always have to be
>> > >>>> thinking about all of the possible ways it could be misused by a
>> user
>> > >>>> (causing them to see hard to diagnose problems).
>> > >>>>
>> > >>> Exactly! That is the point of the discussion really.
>> > >>>
>> > >>>
>> > >>>> In my attempt at a couple methods with builders I feel that I could
>> > >>>> confidently say the user couldn’t really mess it up.
>> > >>>>> // Count
>> > >>>>> KTable<String, Long> count =
>> > >>>>>
>> > kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>> > >>>> The kGroupedStream is reusable and if they attempted to reuse the
>> > Count
>> > >>>> for some reason it would throw an error message saying that a store
>> > >> named
>> > >>>> “my-store” already exists.
>> > >>>>
>> > >>>>
>> > >>> Yes i agree and i think using builders is my preferred pattern.
>> > >>>
>> > >>> Cheers,
>> > >>> Damian
>> > >>>
>> > >>>
>> > >>>> Thanks,
>> > >>>> Kyle
>> > >>>>
>> > >>>> From: Damian Guy
>> > >>>> Sent: Thursday, June 29, 2017 3:59 AM
>> > >>>> To: dev@kafka.apache.org
>> > >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>> > >>>>
>> > >>>> Hi Kyle,
>> > >>>>
>> > >>>> Thanks for your input. Really appreciated.
>> > >>>>
>> > >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>> winkelman.kyle@gmail.com
>> > >
>> > >>>> wrote:
>> > >>>>
>> > >>>>> I like more of a builder pattern even though others have voiced
>> > against
>> > >>>>> it. The reason I like it is because it makes it clear to the user
>> > that
>> > >> a
>> > >>>>> call to KGroupedStream#count will return a KTable not some
>> > intermediate
>> > >>>>> class that I need to undetstand.
>> > >>>>>
>> > >>>> Yes, that makes sense.
>> > >>>>
>> > >>>>
>> > >>>>> When trying to program in the fluent API that has been discussed
>> most
>> > >> it
>> > >>>>> feels difficult to know when you will actually get an object you
>> can
>> > >>>> reuse.
>> > >>>>> What if I make one KGroupedStream that I want to reuse, is it
>> legal
>> > to
>> > >>>>> reuse it or does this approach expect you to call grouped each
>> time?
>> > >>>>
>> > >>>> I'd anticipate that once you have a KGroupedStream you can re-use
>> it
>> > as
>> > >> you
>> > >>>> can today.
>> > >>>>
>> > >>>>
>> > >>>>> This question doesn’t pop into my head at all in the builder
>> pattern
>> > I
>> > >>>>> assume I can reuse everything.
>> > >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
>> > fan
>> > >> of
>> > >>>>> the grouped.
>> > >>>>>
>> > >>>>> Yes, grouped() was more for demonstration and because groupBy()
>> and
>> > >>>> groupByKey() were taken! So i'd imagine the api would actually
>> want to
>> > >> be
>> > >>>> groupByKey(/** no required args***/).withOptionalArg() and
>> > >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>> > >> depends
>> > >>>> on maintaining backward compatibility.
>> > >>>>
>> > >>>>
>> > >>>>> Unfortunately, the below approach would require atleast 2
>> (probably
>> > 3)
>> > >>>>> overloads (one for returning a KTable and one for returning a
>> KTable
>> > >> with
>> > >>>>> Windowed Key, probably would want to split windowed and
>> > sessionwindowed
>> > >>>> for
>> > >>>>> ease of implementation) of each count, reduce, and aggregate.
>> > >>>>> Obviously not exhaustive but enough for you to get the picture.
>> > Count,
>> > >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>> > >> builder:
>> > >>>>> // Count
>> > >>>>> KTable<String, Long> count =
>> > >>>>>
>> > groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>> // Windowed Count
>> > >>>>> KTable<Windowed<String>, Long> windowedCount =
>> > >>>>>
>> > >>
>> > groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>> (10)).withQueryableStoreName("my-windowed-store"));
>> > >>>>> // Session Count
>> > >>>>> KTable<Windowed<String>, Long> sessionCount =
>> > >>>>>
>> > >>
>> > groupedStream.count(Count.sessionWindowed(SessionWindows.
>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>> > >>>>>
>> > >>>> Above and below, i think i'd prefer it to be:
>> > >>>> groupedStream.count(/** non windowed count**/)
>> > >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>> > >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>> > >>>>
>> > >>>>
>> > >>>>
>> > >>>>
>> > >>>>> // Reduce
>> > >>>>> Reducer<Long> reducer;
>> > >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>> > >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>> // Aggregate Windowed with Custom Store
>> > >>>>> Initializer<String> initializer;
>> > >>>>> Aggregator<String, Long, String> aggregator;
>> > >>>>> KTable<Windowed<String>, String> aggregate =
>> > >>>>> groupedStream.aggregate(initializer, aggregator,
>> > >>>>>
>> > >>
>> > Aggregate.windowed(TimeWindows.of(10L).until(10)).
>> withStateStoreSupplier(stateStoreSupplier)));
>> > >>>>> // Cogroup SessionWindowed
>> > >>>>> KTable<String, String> cogrouped =
>> > groupedStream1.cogroup(aggregator1)
>> > >>>>>          .cogroup(groupedStream2, aggregator2)
>> > >>>>>          .aggregate(initializer, aggregator,
>> > >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>> > >>>>> sessionMerger).withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> public class Count {
>> > >>>>>
>> > >>>>>      public static class Windowed extends Count {
>> > >>>>>          private Windows windows;
>> > >>>>>      }
>> > >>>>>      public static class SessionWindowed extends Count {
>> > >>>>>          private SessionWindows sessionWindows;
>> > >>>>>      }
>> > >>>>>
>> > >>>>>      public static Count count();
>> > >>>>>      public static Windowed windowed(Windows windows);
>> > >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
>> > >>>>> sessionWindows);
>> > >>>>>
>> > >>>>>      // All withXXX(...) methods.
>> > >>>>> }
>> > >>>>>
>> > >>>>> public class KGroupedStream {
>> > >>>>>      public KTable<K, Long> count(Count count);
>> > >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
>> > >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>> > >> count);
>> > >>>>> …
>> > >>>>> }
>> > >>>>>
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> From: Guozhang Wang
>> > >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>> > >>>>> To: dev@kafka.apache.org
>> > >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>> > >>>>>
>> > >>>>> I played the current proposal a bit with
>> > >> https://github.com/dguy/kafka/
>> > >>>>> tree/dsl-experiment <
>> > https://github.com/dguy/kafka/tree/dsl-experiment
>> > >>> ,
>> > >>>>> and here are my observations:
>> > >>>>>
>> > >>>>> 1. Personally I prefer
>> > >>>>>
>> > >>>>>      "stream.group(mapper) / stream.groupByKey()"
>> > >>>>>
>> > >>>>> than
>> > >>>>>
>> > >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
>> > >>>>>
>> > >>>>> Since 1) withKeyMapper is not enforced programmatically though it
>> is
>> > >> not
>> > >>>>> "really" optional like others, 2) syntax-wise it reads more
>> natural.
>> > >>>>>
>> > >>>>> I think it is okay to add the APIs in (
>> > >>>>>
>> > >>>>>
>> > >>
>> > https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>> > >>>>> )
>> > >>>>> in KGroupedStream.
>> > >>>>>
>> > >>>>>
>> > >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>> > pass
>> > >> in
>> > >>>>> the most-inner state store supplier (e.g. then one whose get()
>> return
>> > >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
>> > with
>> > >>>>> logging / metrics / etc? I think it would be more useful to only
>> > >> require
>> > >>>>> users pass in the inner state store supplier while specifying
>> > caching /
>> > >>>>> logging through other APIs.
>> > >>>>>
>> > >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>> me:
>> > we
>> > >>>> are
>> > >>>>> allowing users to call other APIs like "withQueryableName"
>> multiple
>> > >> time,
>> > >>>>> but only call "withStateStoreSupplier" only once in the end. Why
>> is
>> > >> that?
>> > >>>>>
>> > >>>>> 3. The current DSL seems to be only for aggregations, what about
>> > joins?
>> > >>>>>
>> > >>>>>
>> > >>>>> 4. I think it is okay to keep the "withLogConfig": for the
>> > >>>>> StateStoreSupplier it will still be user code specifying the
>> topology
>> > >> so
>> > >>>> I
>> > >>>>> do not see there is a big difference.
>> > >>>>>
>> > >>>>>
>> > >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>> the
>> > >>>>> windowed state store supplier to enforce typing?
>> > >>>>>
>> > >>>>>
>> > >>>>> Below are minor ones:
>> > >>>>>
>> > >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>> > >>>>>
>> > >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>> > >> matthias@confluent.io>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>> I see your point about "when to add the processor to the
>> topology".
>> > >>>> That
>> > >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>> > >>>>> topology...
>> > >>>>>> I don't see any problem with having all the withXX() in KTable
>> > >>>> interface
>> > >>>>>> -- but this might be subjective.
>> > >>>>>>
>> > >>>>>>
>> > >>>>>> However, I don't understand your argument about putting
>> aggregate()
>> > >>>>>> after the withXX() -- all the calls to withXX() set optional
>> > >> parameters
>> > >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> > >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>> > might
>> > >>>>>> be quite confusion for developers.
>> > >>>>>>
>> > >>>>>>
>> > >>>>>> -Matthias
>> > >>>>>>
>> > >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>> > >>>>>>>> I also think that mixing optional parameters with configs is a
>> bad
>> > >>>>> idea.
>> > >>>>>>>> Have not proposal for this atm but just wanted to mention it.
>> Hope
>> > >>>> to
>> > >>>>>>>> find some time to come up with something.
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> Yes, i don't like the mix of config either. But the only real
>> > config
>> > >>>>> here
>> > >>>>>>> is the logging config - which we don't really need as it can
>> > already
>> > >>>> be
>> > >>>>>>> done via a custom StateStoreSupplier.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> What I don't like in the current proposal is the
>> > >>>>>>>> .grouped().withKeyMapper() -- the current solution with
>> > >>>> .groupBy(...)
>> > >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>> > >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
>> > find
>> > >>>>>>>> some better names).
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> it could be groupByKey(), groupBy() or something different bt
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> The proposed pattern "chains" grouping and aggregation too
>> close
>> > >>>>>>>> together. I would rather separate both more than less, ie, do
>> into
>> > >>>> the
>> > >>>>>>>> opposite direction.
>> > >>>>>>>>
>> > >>>>>>>> I am also wondering, if we could so something more "fluent".
>> The
>> > >>>>> initial
>> > >>>>>>>> proposal was like:
>> > >>>>>>>>
>> > >>>>>>>>>> groupedStream.count()
>> > >>>>>>>>>>     .withStoreName("name")
>> > >>>>>>>>>>     .withCachingEnabled(false)
>> > >>>>>>>>>>     .withLoggingEnabled(config)
>> > >>>>>>>>>>     .table()
>> > >>>>>>>> The .table() statement in the end was kinda alien.
>> > >>>>>>>>
>> > >>>>>>> I agree, but then all of the withXXX methods need to be on
>> KTable
>> > >>>> which
>> > >>>>>> is
>> > >>>>>>> worse in my opinion. You also need something that is going to
>> > "build"
>> > >>>>> the
>> > >>>>>>> internal processors and add them to the topology.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> The current proposal put the count() into the end -- ie, the
>> > >>>> optional
>> > >>>>>>>> parameter for count() have to specified on the .grouped() call
>> --
>> > >>>> this
>> > >>>>>>>> does not seems to be the best way either.
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> I actually prefer this method as you are building a grouped
>> stream
>> > >>>> that
>> > >>>>>> you
>> > >>>>>>> will aggregate. So
>> > >>>> table.grouped(...).withOptionalStuff().aggregate(..)
>> > >>>>>> etc
>> > >>>>>>> seems natural to me.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> I did not think this through in detail, but can't we just do
>> the
>> > >>>>> initial
>> > >>>>>>>> proposal with the .table() ?
>> > >>>>>>>>
>> > >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>> > >>>>>>>>
>> > >>>>>>>> Each .withXXX(...) return the current KTable and all the
>> > .withXXX()
>> > >>>>> are
>> > >>>>>>>> just added to the KTable interface. Or do I miss anything why
>> this
>> > >>>>> wont'
>> > >>>>>>>> work or any obvious disadvantage?
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> See above.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> -Matthias
>> > >>>>>>>>
>> > >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>> > >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>> > >>>> fluent
>> > >>>>>>>>> approach, but i think it is slightly nicer.
>> > >>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>> in
>> > >>>> the
>> > >>>>>> DSL,
>> > >>>>>>>>> but i think that enabling caching and enabling logging are
>> things
>> > >>>>> that
>> > >>>>>>>>> aren't actually config. I'd probably not add
>> withLogConfig(...)
>> > >>>> (even
>> > >>>>>>>>> though it is below) as this is actually config and we already
>> > have
>> > >>>> a
>> > >>>>>> way
>> > >>>>>>>> of
>> > >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>> the
>> > >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
>> > that
>> > >>>>> is
>> > >>>>>> a
>> > >>>>>>>>> bit of a tedious process for someone that just wants to use
>> the
>> > >>>>> default
>> > >>>>>>>>> storage engine, but not have caching enabled.
>> > >>>>>>>>>
>> > >>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>> to....
>> > If
>> > >>>>> you
>> > >>>>>>>>> want to plug in a custom storage engine and you want it to be
>> > >>>> logged
>> > >>>>>> etc,
>> > >>>>>>>>> you would currently need to implement that yourself. Ideally
>> we
>> > can
>> > >>>>>>>> provide
>> > >>>>>>>>> a way where we will wrap the custom store with logging,
>> metrics,
>> > >>>>> etc. I
>> > >>>>>>>>> need to think about where this fits, it is probably more
>> > >>>> appropriate
>> > >>>>> on
>> > >>>>>>>> the
>> > >>>>>>>>> Stores API.
>> > >>>>>>>>>
>> > >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>> > >>>>>>>>> // count with mapped key
>> > >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>> > >>>>>>>>>          .withKeyMapper(keyMapper)
>> > >>>>>>>>>          .withKeySerde(Serdes.Long())
>> > >>>>>>>>>          .withValueSerde(Serdes.String())
>> > >>>>>>>>>          .withQueryableName("my-store")
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // windowed count
>> > >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>> > >>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-window-store")
>> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // windowed reduce
>> > >>>>>>>>> final Reducer<String> windowedReducer = null;
>> > >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>> > >>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-window-store")
>> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>> > >>>>>>>>>          .reduce(windowedReducer);
>> > >>>>>>>>>
>> > >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>> > >>>>>>>>> final Initializer<Long> init = null;
>> > >>>>>>>>>
>> > >>>>>>>>> // aggregate
>> > >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-aggregate-store")
>> > >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
>> > >>>>>>>>>
>> > >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>> > >>>>>> stateStoreSupplier
>> > >>>>>>>> = null;
>> > >>>>>>>>> // aggregate with custom store
>> > >>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>> stream.grouped()
>> > >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
>> > >>>>>>>>>          .aggregate(aggregator, init);
>> > >>>>>>>>>
>> > >>>>>>>>> // disable caching
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("name")
>> > >>>>>>>>>          .withCachingEnabled(false)
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // disable logging
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("q")
>> > >>>>>>>>>          .withLoggingEnabled(false)
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // override log config
>> > >>>>>>>>> final Reducer<String> reducer = null;
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withLogConfig(Collections.sin
>> gletonMap("segment.size",
>> > >>>>> "10"))
>> > >>>>>>>>>          .reduce(reducer);
>> > >>>>>>>>>
>> > >>>>>>>>>
>> > >>>>>>>>> If anyone wants to play around with this you can find the code
>> > >>>> here:
>> > >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>> > >>>>>>>>>
>> > >>>>>>>>> Note: It won't actually work as most of the methods just
>> return
>> > >>>> null.
>> > >>>>>>>>> Thanks,
>> > >>>>>>>>> Damian
>> > >>>>>>>>>
>> > >>>>>>>>>
>> > >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>> > >>>> wrote:
>> > >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>> both
>> > >>>> are
>> > >>>>>>>> better
>> > >>>>>>>>>> than overload abuse.
>> > >>>>>>>>>>
>> > >>>>>>>>>> The fluent API approach reads better, no mention of builder
>> or
>> > >>>> build
>> > >>>>>>>>>> anywhere. The main downside is that the method signatures
>> are a
>> > >>>>> little
>> > >>>>>>>> less
>> > >>>>>>>>>> clear. By reading the method signature, one doesn't
>> necessarily
>> > >>>>> knows
>> > >>>>>>>> what
>> > >>>>>>>>>> it returns. Also, one needs to figure out the special method
>> > >>>>>> (`table()`
>> > >>>>>>>> in
>> > >>>>>>>>>> this case) that gives you what you actually care about
>> (`KTable`
>> > >>>> in
>> > >>>>>> this
>> > >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>> > >>>>>>>> comparison.
>> > >>>>>>>>>> The builder approach avoids the issues mentioned above, but
>> it
>> > >>>>> doesn't
>> > >>>>>>>> read
>> > >>>>>>>>>> as well.
>> > >>>>>>>>>>
>> > >>>>>>>>>> Ismael
>> > >>>>>>>>>>
>> > >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>> > damian.guy@gmail.com
>> > >>>>>>>> wrote:
>> > >>>>>>>>>>> Hi,
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> I'd like to get a discussion going around some of the API
>> > choices
>> > >>>>>> we've
>> > >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>> > >>>>>> operations
>> > >>>>>>>>>>> (though this could expand).
>> > >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>> API,
>> > >>>> i.e,
>> > >>>>>>>> there
>> > >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>> > >>>> noisy
>> > >>>>>> and
>> > >>>>>>>> i
>> > >>>>>>>>>>> feel it is only going to get worse as we add more optional
>> > >>>> params.
>> > >>>>> In
>> > >>>>>>>>>>> particular we've had some requests to be able to turn
>> caching
>> > >>>> off,
>> > >>>>> or
>> > >>>>>>>>>>> change log configs,  on a per operator basis (note this can
>> be
>> > >>>> done
>> > >>>>>> now
>> > >>>>>>>>>> if
>> > >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>> > >>>>> cumbersome).
>> > >>>>>>>>>>> So this is a bit of an open question. How can we change the
>> DSL
>> > >>>>>>>> overloads
>> > >>>>>>>>>>> so that it flows, is simple to use and understand, and is
>> > easily
>> > >>>>>>>> extended
>> > >>>>>>>>>>> in the future?
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> One option would be to use a fluent API approach for
>> providing
>> > >>>> the
>> > >>>>>>>>>> optional
>> > >>>>>>>>>>> params, so something like this:
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> groupedStream.count()
>> > >>>>>>>>>>>     .withStoreName("name")
>> > >>>>>>>>>>>     .withCachingEnabled(false)
>> > >>>>>>>>>>>     .withLoggingEnabled(config)
>> > >>>>>>>>>>>     .table()
>> > >>>>>>>>>>>
>> > >>>>>>>>>>>
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Another option would be to provide a Builder to the count
>> > method,
>> > >>>>> so
>> > >>>>>> it
>> > >>>>>>>>>>> would look something like this:
>> > >>>>>>>>>>> groupedStream.count(new
>> > >>>>>>>>>>> CountBuilder("storeName").with
>> CachingEnabled(false).build())
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>> you
>> > on
>> > >>>>>>>> about!
>> > >>>>>>>>>>> The above has focussed on state store related overloads, but
>> > the
>> > >>>>> same
>> > >>>>>>>>>> ideas
>> > >>>>>>>>>>> could  be applied to joins etc, where we presently have many
>> > join
>> > >>>>>>>> methods
>> > >>>>>>>>>>> and many overloads.
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Thanks,
>> > >>>>>>>>>>> Damian
>> > >>>>>>>>>>>
>> > >>>>>>>>
>> > >>>>>>
>> > >>>>>
>> > >>>>> --
>> > >>>>> -- Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>
>> > >>
>> >
>> >
>>
>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
I messed the indentation on github code repos; this would be easier to read:

https://codeshare.io/GLWW8K


Guozhang


On Fri, Jul 7, 2017 at 1:30 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Hi Damian / Kyle,
>
> I think I agree with you guys about the pros / cons of using the builder
> pattern v.s. using some "secondary classes". And I'm thinking if we can
> take a "mid" manner between these two. I spent some time with a slight
> different approach from Damian's current proposal:
>
> https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/
> java/org/apache/kafka/streams/RefactoredAPIs.java
>
> The key idea is to tolerate the final "table()" or "stream()" function to
> "upgrade" from the secondary classes to the first citizen classes, while
> having all the specs inside this function. Also this proposal includes some
> other refactoring that people have been discussed about for the builder to
> reduce the overloaded functions as well. WDYT?
>
>
> Guozhang
>
>
> On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:
>
>> Hi Jan,
>>
>> Thanks very much for the input.
>>
>> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com>
>> wrote:
>>
>> > Hi Damian,
>> >
>> > I do see your point of something needs to change. But I fully agree with
>> > Gouzhang when he says.
>> > ---
>> >
>> > But since this is a incompatibility change, and we are going to remove
>> the
>> > compatibility annotations soon it means we only have one chance and we
>> > really have to make it right.
>> > ----
>> >
>> >
>> I think we all agree on this one! Hence the discussion.
>>
>>
>> > I fear all suggestions do not go far enough to become something that
>> will
>> > carry on for very much longer.
>> > I am currently working on KAFKA-3705 and try to find the most easy way
>> for
>> > the user to give me all the required functionality. The easiest
>> interface I
>> > could come up so far can be looked at here.
>> >
>> >
>> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> kafka/streams/kstream/internals/KTableImpl.java#L622
>> >
>> >
>> And its already horribly complicated. I am currently unable to find the
>> > right abstraction level to have everything falling into place
>> naturally. To
>> > be honest I already think introducing
>> >
>> >
>> To be fair that is not a particularly easy problem to solve!
>>
>>
>> >
>> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2
>> de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/
>> kafka/streams/kstream/internals/KTableImpl.java#L493
>> >
>> > was unideal and makes everything a mess.
>>
>>
>> I'm not sure i agree that it makes everything a mess, but It could have
>> been done differently.
>>
>> The JoinType:Whatever is also not really flexible. 2 things come to my
>> mind:
>> >
>> > 1. I don't think we should rule out config based decisions say configs
>> like
>> >         streams.$applicationID.joins.$joinname.conf = value
>> >
>>
>> Is this just for config? Or are you suggesting that we could somehow
>> "code"
>> the join in a config file?
>>
>>
>> > This can allow for tremendous changes without single API change and IMO
>> it
>> > was not considered enough yet.
>> >
>> > 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
>> > example can be used to implement different join types as the user
>> wishes.
>> >
>>
>> Do you have an example of how this might look?
>>
>>
>> > As Gouzhang said: stopping to break users is very important.
>>
>>
>> Of course. We want to make it as easy as possible for people to use
>> streams.
>>
>>
>> especially with this changes + All the plans I sadly only have in my head
>> > but hopefully the first link can give a glimpse.
>> >
>> > Thanks for preparing the examples made it way clearer to me what exactly
>> > we are talking about. I would argue to go a bit slower and more
>> carefull on
>> > this one. At some point we need to get it right. Peeking over to the
>> hadoop
>> > guys with their hughe userbase. Config files really work well for them.
>> >
>> > Best Jan
>> >
>> >
>> >
>> >
>> >
>> > On 30.06.2017 09:31, Damian Guy wrote:
>> > > Thanks Matthias
>> > >
>> > > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> > >
>> > >> I am just catching up on this thread, so sorry for the long email in
>> > >> advance... Also, it's to some extend a dump of thoughts and not
>> always a
>> > >> clear proposal. Still need to think about this in more detail. But
>> maybe
>> > >> it helps other to get new ideas :)
>> > >>
>> > >>
>> > >>>> However, I don't understand your argument about putting aggregate()
>> > >>>> after the withXX() -- all the calls to withXX() set optional
>> > parameters
>> > >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> > >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>> might
>> > >>>> be quite confusion for developers.
>> > >>>>
>> > >>>>
>> > >>> I see what you are saying, but the grouped stream is effectively a
>> > no-op
>> > >>> until you call one of the aggregate/count/reduce etc functions. So
>> the
>> > >>> optional params are ones that are applicable to any of the
>> operations
>> > you
>> > >>> can perform on this grouped stream. Then the final
>> > >>> count()/reduce()/aggregate() call has any of the params that are
>> > >>> required/specific to that function.
>> > >>>
>> > >> I understand your argument, but you don't share the conclusion. If we
>> > >> need a "final/terminal" call, the better way might be
>> > >>
>> > >> .groupBy().count().withXX().build()
>> > >>
>> > >> (with a better name for build() though)
>> > >>
>> > >>
>> > > The point is that all the other calls, i.e,withBlah, windowed, etc
>> apply
>> > > too all the aggregate functions. The terminal call being the actual
>> type
>> > of
>> > > aggregation you want to do. I personally find this more natural than
>> > > groupBy().count().withBlah().build()
>> > >
>> > >
>> > >>> groupedStream.count(/** non windowed count**/)
>> > >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>> > >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>> > >>
>> > >> I like this. However, I don't see a reason to have windowed() and
>> > >> sessionWindowed(). We should have one top-level `Windows` interface
>> that
>> > >> both `TimeWindows` and `SessionWindows` implement and just have a
>> single
>> > >> windowed() method that accepts all `Windows`. (I did not like the
>> > >> separation of `SessionWindows` in the first place, and this seems to
>> be
>> > >> an opportunity to clean this up. It was hard to change when we
>> > >> introduced session windows)
>> > >>
>> > > Yes - true we should look into that.
>> > >
>> > >
>> > >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>> > >> might also want to use windowBy() (instead of windowed()). Not sure
>> how
>> > >> important this is, but it seems to be inconsistent otherwise.
>> > >>
>> > >>
>> > > Makes sense
>> > >
>> > >
>> > >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
>> think,
>> > >> defining an inner/left/outer join is not an optional argument but a
>> > >> first class concept and should have a proper representation in the
>> API
>> > >> (like the current methods join(), leftJoin, outerJoin()).
>> > >>
>> > >>
>> > > Yep, i did originally have it as a required param and maybe that is
>> what
>> > we
>> > > go with. It could have a default, but maybe that is confusing.
>> > >
>> > >
>> > >
>> > >> About the two join API proposals, the second one has too much boiler
>> > >> plate code for my taste. Also, the actual join() operator has only
>> one
>> > >> argument what is weird to me, as in my thinking process, the main
>> > >> operator call, should have one parameter per mandatory argument but
>> your
>> > >> proposal put the mandatory arguments into Joins.streamStreamJoin()
>> call.
>> > >> This is far from intuitive IMHO.
>> > >>
>> > >>
>> > > This is the builder pattern, you only need one param as the builder
>> has
>> > > captured all of the required and optional arguments.
>> > >
>> > >
>> > >> The first join proposal also seems to align better with the pattern
>> > >> suggested for aggregations and having the same pattern for all
>> operators
>> > >> is important (as you stated already).
>> > >>
>> > >>
>> > > This is why i offered two alternatives as i started out with. 1 is the
>> > > builder pattern, the other is the more fluent pattern.
>> > >
>> > >
>> > >>
>> > >> Coming back to the config vs optional parameter. What about having a
>> > >> method withConfig[s](...) that allow to put in the configuration?
>> > >>
>> > >>
>> > > Sure, it is currently called withLogConfig() as that is the only thing
>> > that
>> > > is really config.
>> > >
>> > >
>> > >> This also raises the question if until() is a windows property?
>> > >> Actually, until() seems to be a configuration parameter and thus,
>> should
>> > >> not not have it's own method.
>> > >>
>> > >>
>> > > Hmmm, i don't agree. Until is a property of the window. It is going
>> to be
>> > > potentially different for every window operation you do in a streams
>> app.
>> > >
>> > >
>> > >>
>> > >> Browsing throw your example DSL branch, I also saw this one:
>> > >>
>> > >>> final KTable<Windowed<String>, Long> windowed>
>> > >>   groupedStream.counting()
>> > >>>                   .windowed(TimeWindows.of(10L).until(10))
>> > >>>                   .table();
>> > >> This is an interesting idea, and it remind my on some feedback about
>> "I
>> > >> wanted to count a stream, but there was no count() method -- I first
>> > >> needed to figure out, that I need to group the stream first to be
>> able
>> > >> to count it. It does make sense in hindsight but was not obvious in
>> the
>> > >> beginning". Thus, carrying out this thought, we could also do the
>> > >> following:
>> > >>
>> > >> stream.count().groupedBy().windowedBy().table();
>> > >>
>> > >> -> Note, I use "grouped" and "windowed" instead of imperative here,
>> as
>> > >> it comes after the count()
>> > >>
>> > >> This would be more consistent than your proposal (that has grouping
>> > >> before but windowing after count()). It might even allow us to enrich
>> > >> the API with a some syntactic sugar like `stream.count().table()` to
>> get
>> > >> the overall count of all records (this would obviously not scale,
>> but we
>> > >> could support it -- if not now, maybe later).
>> > >>
>> > >>
>> > > I guess i'd prefer
>> > > stream.groupBy().windowBy().count()
>> > > stream.groupBy().windowBy().reduce()
>> > > stream.groupBy().count()
>> > >
>> > > As i said above, everything that happens before the final aggregate
>> call
>> > > can be applied to any of them. So it makes sense to me to do those
>> things
>> > > ahead of the final aggregate call.
>> > >
>> > >
>> > >> Last about builder pattern. I am convinced that we need some
>> "terminal"
>> > >> operator/method that tells us when to add the processor to the
>> topology.
>> > >> But I don't see the need for a plain builder pattern that feels
>> alien to
>> > >> me (see my argument about the second join proposal). Using .stream()
>> /
>> > >> .table() as use in many examples might work. But maybe a more generic
>> > >> name that we can use in all places like build() or apply() might
>> also be
>> > >> an option.
>> > >>
>> > >>
>> > > Sure, a generic name might be ok.
>> > >
>> > >
>> > >
>> > >
>> > >> -Matthias
>> > >>
>> > >>
>> > >>
>> > >> On 6/29/17 7:37 AM, Damian Guy wrote:
>> > >>> Thanks Kyle.
>> > >>>
>> > >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
>> winkelman.kyle@gmail.com>
>> > >>> wrote:
>> > >>>
>> > >>>> Hi Damian,
>> > >>>>
>> > >>>>>>>> When trying to program in the fluent API that has been
>> discussed
>> > >> most
>> > >>>> it
>> > >>>>>>>> feels difficult to know when you will actually get an object
>> you
>> > can
>> > >>>> reuse.
>> > >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>> > legal
>> > >> to
>> > >>>>>>>> reuse it or does this approach expect you to call grouped each
>> > time?
>> > >>>>>> I'd anticipate that once you have a KGroupedStream you can
>> re-use it
>> > >> as
>> > >>>> you
>> > >>>>>> can today.
>> > >>>> You said it yourself in another post that the grouped stream is
>> > >>>> effectively a no-op until a count, reduce, or aggregate. The way I
>> see
>> > >> it
>> > >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>> > >> because
>> > >>>> most of this fluent api would continue returning this (this being
>> the
>> > >>>> builder object currently being manipulated).
>> > >>> So, if you ever store a reference to anything but KStreams and
>> KTables
>> > >> and
>> > >>>> you use it in two different ways then its possible you make
>> > conflicting
>> > >>>> withXXX() calls on the same builder.
>> > >>>>
>> > >>>>
>> > >>> No necessarily true. It could return a new instance of the builder,
>> > i.e.,
>> > >>> the builders being immutable. So if you held a reference to the
>> builder
>> > >> it
>> > >>> would always be the same as it was when it was created.
>> > >>>
>> > >>>
>> > >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
>> kStream.grouped();
>> > >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>> > >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>> > >>>>
>> > >>>> I’ll admit that this shouldn’t happen but some user is going to do
>> it
>> > >>>> eventually…
>> > >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>> > would
>> > >>>> most likely be equivalent to the version withDeclaredSerdes. One
>> work
>> > >>>> around would be to always make copies of the config objects you are
>> > >>>> building, but this approach has its own problem because now we
>> have to
>> > >>>> identify which configs are equivalent so we don’t create repeated
>> > >>>> processors.
>> > >>>>
>> > >>>> The point of this long winded example is that we always have to be
>> > >>>> thinking about all of the possible ways it could be misused by a
>> user
>> > >>>> (causing them to see hard to diagnose problems).
>> > >>>>
>> > >>> Exactly! That is the point of the discussion really.
>> > >>>
>> > >>>
>> > >>>> In my attempt at a couple methods with builders I feel that I could
>> > >>>> confidently say the user couldn’t really mess it up.
>> > >>>>> // Count
>> > >>>>> KTable<String, Long> count =
>> > >>>>>
>> > kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>> > >>>> The kGroupedStream is reusable and if they attempted to reuse the
>> > Count
>> > >>>> for some reason it would throw an error message saying that a store
>> > >> named
>> > >>>> “my-store” already exists.
>> > >>>>
>> > >>>>
>> > >>> Yes i agree and i think using builders is my preferred pattern.
>> > >>>
>> > >>> Cheers,
>> > >>> Damian
>> > >>>
>> > >>>
>> > >>>> Thanks,
>> > >>>> Kyle
>> > >>>>
>> > >>>> From: Damian Guy
>> > >>>> Sent: Thursday, June 29, 2017 3:59 AM
>> > >>>> To: dev@kafka.apache.org
>> > >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>> > >>>>
>> > >>>> Hi Kyle,
>> > >>>>
>> > >>>> Thanks for your input. Really appreciated.
>> > >>>>
>> > >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
>> winkelman.kyle@gmail.com
>> > >
>> > >>>> wrote:
>> > >>>>
>> > >>>>> I like more of a builder pattern even though others have voiced
>> > against
>> > >>>>> it. The reason I like it is because it makes it clear to the user
>> > that
>> > >> a
>> > >>>>> call to KGroupedStream#count will return a KTable not some
>> > intermediate
>> > >>>>> class that I need to undetstand.
>> > >>>>>
>> > >>>> Yes, that makes sense.
>> > >>>>
>> > >>>>
>> > >>>>> When trying to program in the fluent API that has been discussed
>> most
>> > >> it
>> > >>>>> feels difficult to know when you will actually get an object you
>> can
>> > >>>> reuse.
>> > >>>>> What if I make one KGroupedStream that I want to reuse, is it
>> legal
>> > to
>> > >>>>> reuse it or does this approach expect you to call grouped each
>> time?
>> > >>>>
>> > >>>> I'd anticipate that once you have a KGroupedStream you can re-use
>> it
>> > as
>> > >> you
>> > >>>> can today.
>> > >>>>
>> > >>>>
>> > >>>>> This question doesn’t pop into my head at all in the builder
>> pattern
>> > I
>> > >>>>> assume I can reuse everything.
>> > >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
>> > fan
>> > >> of
>> > >>>>> the grouped.
>> > >>>>>
>> > >>>>> Yes, grouped() was more for demonstration and because groupBy()
>> and
>> > >>>> groupByKey() were taken! So i'd imagine the api would actually
>> want to
>> > >> be
>> > >>>> groupByKey(/** no required args***/).withOptionalArg() and
>> > >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>> > >> depends
>> > >>>> on maintaining backward compatibility.
>> > >>>>
>> > >>>>
>> > >>>>> Unfortunately, the below approach would require atleast 2
>> (probably
>> > 3)
>> > >>>>> overloads (one for returning a KTable and one for returning a
>> KTable
>> > >> with
>> > >>>>> Windowed Key, probably would want to split windowed and
>> > sessionwindowed
>> > >>>> for
>> > >>>>> ease of implementation) of each count, reduce, and aggregate.
>> > >>>>> Obviously not exhaustive but enough for you to get the picture.
>> > Count,
>> > >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>> > >> builder:
>> > >>>>> // Count
>> > >>>>> KTable<String, Long> count =
>> > >>>>>
>> > groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>> // Windowed Count
>> > >>>>> KTable<Windowed<String>, Long> windowedCount =
>> > >>>>>
>> > >>
>> > groupedStream.count(Count.windowed(TimeWindows.of(10L).until
>> (10)).withQueryableStoreName("my-windowed-store"));
>> > >>>>> // Session Count
>> > >>>>> KTable<Windowed<String>, Long> sessionCount =
>> > >>>>>
>> > >>
>> > groupedStream.count(Count.sessionWindowed(SessionWindows.
>> with(10L)).withQueryableStoreName("my-session-windowed-store"));
>> > >>>>>
>> > >>>> Above and below, i think i'd prefer it to be:
>> > >>>> groupedStream.count(/** non windowed count**/)
>> > >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>> > >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>> > >>>>
>> > >>>>
>> > >>>>
>> > >>>>
>> > >>>>> // Reduce
>> > >>>>> Reducer<Long> reducer;
>> > >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>> > >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>> // Aggregate Windowed with Custom Store
>> > >>>>> Initializer<String> initializer;
>> > >>>>> Aggregator<String, Long, String> aggregator;
>> > >>>>> KTable<Windowed<String>, String> aggregate =
>> > >>>>> groupedStream.aggregate(initializer, aggregator,
>> > >>>>>
>> > >>
>> > Aggregate.windowed(TimeWindows.of(10L).until(10)).
>> withStateStoreSupplier(stateStoreSupplier)));
>> > >>>>> // Cogroup SessionWindowed
>> > >>>>> KTable<String, String> cogrouped =
>> > groupedStream1.cogroup(aggregator1)
>> > >>>>>          .cogroup(groupedStream2, aggregator2)
>> > >>>>>          .aggregate(initializer, aggregator,
>> > >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>> > >>>>> sessionMerger).withQueryableStoreName("my-store"));
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> public class Count {
>> > >>>>>
>> > >>>>>      public static class Windowed extends Count {
>> > >>>>>          private Windows windows;
>> > >>>>>      }
>> > >>>>>      public static class SessionWindowed extends Count {
>> > >>>>>          private SessionWindows sessionWindows;
>> > >>>>>      }
>> > >>>>>
>> > >>>>>      public static Count count();
>> > >>>>>      public static Windowed windowed(Windows windows);
>> > >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
>> > >>>>> sessionWindows);
>> > >>>>>
>> > >>>>>      // All withXXX(...) methods.
>> > >>>>> }
>> > >>>>>
>> > >>>>> public class KGroupedStream {
>> > >>>>>      public KTable<K, Long> count(Count count);
>> > >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
>> > >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>> > >> count);
>> > >>>>> …
>> > >>>>> }
>> > >>>>>
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> From: Guozhang Wang
>> > >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>> > >>>>> To: dev@kafka.apache.org
>> > >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>> > >>>>>
>> > >>>>> I played the current proposal a bit with
>> > >> https://github.com/dguy/kafka/
>> > >>>>> tree/dsl-experiment <
>> > https://github.com/dguy/kafka/tree/dsl-experiment
>> > >>> ,
>> > >>>>> and here are my observations:
>> > >>>>>
>> > >>>>> 1. Personally I prefer
>> > >>>>>
>> > >>>>>      "stream.group(mapper) / stream.groupByKey()"
>> > >>>>>
>> > >>>>> than
>> > >>>>>
>> > >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
>> > >>>>>
>> > >>>>> Since 1) withKeyMapper is not enforced programmatically though it
>> is
>> > >> not
>> > >>>>> "really" optional like others, 2) syntax-wise it reads more
>> natural.
>> > >>>>>
>> > >>>>> I think it is okay to add the APIs in (
>> > >>>>>
>> > >>>>>
>> > >>
>> > https://github.com/dguy/kafka/blob/dsl-experiment/streams/sr
>> c/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>> > >>>>> )
>> > >>>>> in KGroupedStream.
>> > >>>>>
>> > >>>>>
>> > >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>> > pass
>> > >> in
>> > >>>>> the most-inner state store supplier (e.g. then one whose get()
>> return
>> > >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
>> > with
>> > >>>>> logging / metrics / etc? I think it would be more useful to only
>> > >> require
>> > >>>>> users pass in the inner state store supplier while specifying
>> > caching /
>> > >>>>> logging through other APIs.
>> > >>>>>
>> > >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
>> me:
>> > we
>> > >>>> are
>> > >>>>> allowing users to call other APIs like "withQueryableName"
>> multiple
>> > >> time,
>> > >>>>> but only call "withStateStoreSupplier" only once in the end. Why
>> is
>> > >> that?
>> > >>>>>
>> > >>>>> 3. The current DSL seems to be only for aggregations, what about
>> > joins?
>> > >>>>>
>> > >>>>>
>> > >>>>> 4. I think it is okay to keep the "withLogConfig": for the
>> > >>>>> StateStoreSupplier it will still be user code specifying the
>> topology
>> > >> so
>> > >>>> I
>> > >>>>> do not see there is a big difference.
>> > >>>>>
>> > >>>>>
>> > >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
>> the
>> > >>>>> windowed state store supplier to enforce typing?
>> > >>>>>
>> > >>>>>
>> > >>>>> Below are minor ones:
>> > >>>>>
>> > >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>> > >>>>>
>> > >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>> > >> matthias@confluent.io>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>> I see your point about "when to add the processor to the
>> topology".
>> > >>>> That
>> > >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>> > >>>>> topology...
>> > >>>>>> I don't see any problem with having all the withXX() in KTable
>> > >>>> interface
>> > >>>>>> -- but this might be subjective.
>> > >>>>>>
>> > >>>>>>
>> > >>>>>> However, I don't understand your argument about putting
>> aggregate()
>> > >>>>>> after the withXX() -- all the calls to withXX() set optional
>> > >> parameters
>> > >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> > >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>> > might
>> > >>>>>> be quite confusion for developers.
>> > >>>>>>
>> > >>>>>>
>> > >>>>>> -Matthias
>> > >>>>>>
>> > >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>> > >>>>>>>> I also think that mixing optional parameters with configs is a
>> bad
>> > >>>>> idea.
>> > >>>>>>>> Have not proposal for this atm but just wanted to mention it.
>> Hope
>> > >>>> to
>> > >>>>>>>> find some time to come up with something.
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> Yes, i don't like the mix of config either. But the only real
>> > config
>> > >>>>> here
>> > >>>>>>> is the logging config - which we don't really need as it can
>> > already
>> > >>>> be
>> > >>>>>>> done via a custom StateStoreSupplier.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> What I don't like in the current proposal is the
>> > >>>>>>>> .grouped().withKeyMapper() -- the current solution with
>> > >>>> .groupBy(...)
>> > >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>> > >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
>> > find
>> > >>>>>>>> some better names).
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> it could be groupByKey(), groupBy() or something different bt
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> The proposed pattern "chains" grouping and aggregation too
>> close
>> > >>>>>>>> together. I would rather separate both more than less, ie, do
>> into
>> > >>>> the
>> > >>>>>>>> opposite direction.
>> > >>>>>>>>
>> > >>>>>>>> I am also wondering, if we could so something more "fluent".
>> The
>> > >>>>> initial
>> > >>>>>>>> proposal was like:
>> > >>>>>>>>
>> > >>>>>>>>>> groupedStream.count()
>> > >>>>>>>>>>     .withStoreName("name")
>> > >>>>>>>>>>     .withCachingEnabled(false)
>> > >>>>>>>>>>     .withLoggingEnabled(config)
>> > >>>>>>>>>>     .table()
>> > >>>>>>>> The .table() statement in the end was kinda alien.
>> > >>>>>>>>
>> > >>>>>>> I agree, but then all of the withXXX methods need to be on
>> KTable
>> > >>>> which
>> > >>>>>> is
>> > >>>>>>> worse in my opinion. You also need something that is going to
>> > "build"
>> > >>>>> the
>> > >>>>>>> internal processors and add them to the topology.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> The current proposal put the count() into the end -- ie, the
>> > >>>> optional
>> > >>>>>>>> parameter for count() have to specified on the .grouped() call
>> --
>> > >>>> this
>> > >>>>>>>> does not seems to be the best way either.
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> I actually prefer this method as you are building a grouped
>> stream
>> > >>>> that
>> > >>>>>> you
>> > >>>>>>> will aggregate. So
>> > >>>> table.grouped(...).withOptionalStuff().aggregate(..)
>> > >>>>>> etc
>> > >>>>>>> seems natural to me.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> I did not think this through in detail, but can't we just do
>> the
>> > >>>>> initial
>> > >>>>>>>> proposal with the .table() ?
>> > >>>>>>>>
>> > >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>> > >>>>>>>>
>> > >>>>>>>> Each .withXXX(...) return the current KTable and all the
>> > .withXXX()
>> > >>>>> are
>> > >>>>>>>> just added to the KTable interface. Or do I miss anything why
>> this
>> > >>>>> wont'
>> > >>>>>>>> work or any obvious disadvantage?
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>> See above.
>> > >>>>>>>
>> > >>>>>>>
>> > >>>>>>>> -Matthias
>> > >>>>>>>>
>> > >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>> > >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>> > >>>> fluent
>> > >>>>>>>>> approach, but i think it is slightly nicer.
>> > >>>>>>>>> I agree with some of what Eno said about mixing configy stuff
>> in
>> > >>>> the
>> > >>>>>> DSL,
>> > >>>>>>>>> but i think that enabling caching and enabling logging are
>> things
>> > >>>>> that
>> > >>>>>>>>> aren't actually config. I'd probably not add
>> withLogConfig(...)
>> > >>>> (even
>> > >>>>>>>>> though it is below) as this is actually config and we already
>> > have
>> > >>>> a
>> > >>>>>> way
>> > >>>>>>>> of
>> > >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
>> the
>> > >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
>> > that
>> > >>>>> is
>> > >>>>>> a
>> > >>>>>>>>> bit of a tedious process for someone that just wants to use
>> the
>> > >>>>> default
>> > >>>>>>>>> storage engine, but not have caching enabled.
>> > >>>>>>>>>
>> > >>>>>>>>> There is also an orthogonal concern that Guozhang alluded
>> to....
>> > If
>> > >>>>> you
>> > >>>>>>>>> want to plug in a custom storage engine and you want it to be
>> > >>>> logged
>> > >>>>>> etc,
>> > >>>>>>>>> you would currently need to implement that yourself. Ideally
>> we
>> > can
>> > >>>>>>>> provide
>> > >>>>>>>>> a way where we will wrap the custom store with logging,
>> metrics,
>> > >>>>> etc. I
>> > >>>>>>>>> need to think about where this fits, it is probably more
>> > >>>> appropriate
>> > >>>>> on
>> > >>>>>>>> the
>> > >>>>>>>>> Stores API.
>> > >>>>>>>>>
>> > >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>> > >>>>>>>>> // count with mapped key
>> > >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>> > >>>>>>>>>          .withKeyMapper(keyMapper)
>> > >>>>>>>>>          .withKeySerde(Serdes.Long())
>> > >>>>>>>>>          .withValueSerde(Serdes.String())
>> > >>>>>>>>>          .withQueryableName("my-store")
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // windowed count
>> > >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>> > >>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-window-store")
>> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // windowed reduce
>> > >>>>>>>>> final Reducer<String> windowedReducer = null;
>> > >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>> > >>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-window-store")
>> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>> > >>>>>>>>>          .reduce(windowedReducer);
>> > >>>>>>>>>
>> > >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>> > >>>>>>>>> final Initializer<Long> init = null;
>> > >>>>>>>>>
>> > >>>>>>>>> // aggregate
>> > >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>> > >>>>>>>>>          .withQueryableName("my-aggregate-store")
>> > >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
>> > >>>>>>>>>
>> > >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>> > >>>>>> stateStoreSupplier
>> > >>>>>>>> = null;
>> > >>>>>>>>> // aggregate with custom store
>> > >>>>>>>>> final KTable<String, Long> aggWithCustomStore =
>> stream.grouped()
>> > >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
>> > >>>>>>>>>          .aggregate(aggregator, init);
>> > >>>>>>>>>
>> > >>>>>>>>> // disable caching
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("name")
>> > >>>>>>>>>          .withCachingEnabled(false)
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // disable logging
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withQueryableName("q")
>> > >>>>>>>>>          .withLoggingEnabled(false)
>> > >>>>>>>>>          .count();
>> > >>>>>>>>>
>> > >>>>>>>>> // override log config
>> > >>>>>>>>> final Reducer<String> reducer = null;
>> > >>>>>>>>> stream.grouped()
>> > >>>>>>>>>          .withLogConfig(Collections.sin
>> gletonMap("segment.size",
>> > >>>>> "10"))
>> > >>>>>>>>>          .reduce(reducer);
>> > >>>>>>>>>
>> > >>>>>>>>>
>> > >>>>>>>>> If anyone wants to play around with this you can find the code
>> > >>>> here:
>> > >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>> > >>>>>>>>>
>> > >>>>>>>>> Note: It won't actually work as most of the methods just
>> return
>> > >>>> null.
>> > >>>>>>>>> Thanks,
>> > >>>>>>>>> Damian
>> > >>>>>>>>>
>> > >>>>>>>>>
>> > >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>> > >>>> wrote:
>> > >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
>> both
>> > >>>> are
>> > >>>>>>>> better
>> > >>>>>>>>>> than overload abuse.
>> > >>>>>>>>>>
>> > >>>>>>>>>> The fluent API approach reads better, no mention of builder
>> or
>> > >>>> build
>> > >>>>>>>>>> anywhere. The main downside is that the method signatures
>> are a
>> > >>>>> little
>> > >>>>>>>> less
>> > >>>>>>>>>> clear. By reading the method signature, one doesn't
>> necessarily
>> > >>>>> knows
>> > >>>>>>>> what
>> > >>>>>>>>>> it returns. Also, one needs to figure out the special method
>> > >>>>>> (`table()`
>> > >>>>>>>> in
>> > >>>>>>>>>> this case) that gives you what you actually care about
>> (`KTable`
>> > >>>> in
>> > >>>>>> this
>> > >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>> > >>>>>>>> comparison.
>> > >>>>>>>>>> The builder approach avoids the issues mentioned above, but
>> it
>> > >>>>> doesn't
>> > >>>>>>>> read
>> > >>>>>>>>>> as well.
>> > >>>>>>>>>>
>> > >>>>>>>>>> Ismael
>> > >>>>>>>>>>
>> > >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>> > damian.guy@gmail.com
>> > >>>>>>>> wrote:
>> > >>>>>>>>>>> Hi,
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> I'd like to get a discussion going around some of the API
>> > choices
>> > >>>>>> we've
>> > >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>> > >>>>>> operations
>> > >>>>>>>>>>> (though this could expand).
>> > >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
>> API,
>> > >>>> i.e,
>> > >>>>>>>> there
>> > >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>> > >>>> noisy
>> > >>>>>> and
>> > >>>>>>>> i
>> > >>>>>>>>>>> feel it is only going to get worse as we add more optional
>> > >>>> params.
>> > >>>>> In
>> > >>>>>>>>>>> particular we've had some requests to be able to turn
>> caching
>> > >>>> off,
>> > >>>>> or
>> > >>>>>>>>>>> change log configs,  on a per operator basis (note this can
>> be
>> > >>>> done
>> > >>>>>> now
>> > >>>>>>>>>> if
>> > >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>> > >>>>> cumbersome).
>> > >>>>>>>>>>> So this is a bit of an open question. How can we change the
>> DSL
>> > >>>>>>>> overloads
>> > >>>>>>>>>>> so that it flows, is simple to use and understand, and is
>> > easily
>> > >>>>>>>> extended
>> > >>>>>>>>>>> in the future?
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> One option would be to use a fluent API approach for
>> providing
>> > >>>> the
>> > >>>>>>>>>> optional
>> > >>>>>>>>>>> params, so something like this:
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> groupedStream.count()
>> > >>>>>>>>>>>     .withStoreName("name")
>> > >>>>>>>>>>>     .withCachingEnabled(false)
>> > >>>>>>>>>>>     .withLoggingEnabled(config)
>> > >>>>>>>>>>>     .table()
>> > >>>>>>>>>>>
>> > >>>>>>>>>>>
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Another option would be to provide a Builder to the count
>> > method,
>> > >>>>> so
>> > >>>>>> it
>> > >>>>>>>>>>> would look something like this:
>> > >>>>>>>>>>> groupedStream.count(new
>> > >>>>>>>>>>> CountBuilder("storeName").with
>> CachingEnabled(false).build())
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Another option is to say: Hey we don't need this, what are
>> you
>> > on
>> > >>>>>>>> about!
>> > >>>>>>>>>>> The above has focussed on state store related overloads, but
>> > the
>> > >>>>> same
>> > >>>>>>>>>> ideas
>> > >>>>>>>>>>> could  be applied to joins etc, where we presently have many
>> > join
>> > >>>>>>>> methods
>> > >>>>>>>>>>> and many overloads.
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>> > >>>>>>>>>>>
>> > >>>>>>>>>>> Thanks,
>> > >>>>>>>>>>> Damian
>> > >>>>>>>>>>>
>> > >>>>>>>>
>> > >>>>>>
>> > >>>>>
>> > >>>>> --
>> > >>>>> -- Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>
>> > >>
>> >
>> >
>>
>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Damian / Kyle,

I think I agree with you guys about the pros / cons of using the builder
pattern v.s. using some "secondary classes". And I'm thinking if we can
take a "mid" manner between these two. I spent some time with a slight
different approach from Damian's current proposal:

https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/java/org/apache/kafka/streams/RefactoredAPIs.java

The key idea is to tolerate the final "table()" or "stream()" function to
"upgrade" from the secondary classes to the first citizen classes, while
having all the specs inside this function. Also this proposal includes some
other refactoring that people have been discussed about for the builder to
reduce the overloaded functions as well. WDYT?


Guozhang


On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:

> Hi Jan,
>
> Thanks very much for the input.
>
> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com> wrote:
>
> > Hi Damian,
> >
> > I do see your point of something needs to change. But I fully agree with
> > Gouzhang when he says.
> > ---
> >
> > But since this is a incompatibility change, and we are going to remove
> the
> > compatibility annotations soon it means we only have one chance and we
> > really have to make it right.
> > ----
> >
> >
> I think we all agree on this one! Hence the discussion.
>
>
> > I fear all suggestions do not go far enough to become something that will
> > carry on for very much longer.
> > I am currently working on KAFKA-3705 and try to find the most easy way
> for
> > the user to give me all the required functionality. The easiest
> interface I
> > could come up so far can be looked at here.
> >
> >
> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125
> ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/
> kstream/internals/KTableImpl.java#L622
> >
> >
> And its already horribly complicated. I am currently unable to find the
> > right abstraction level to have everything falling into place naturally.
> To
> > be honest I already think introducing
> >
> >
> To be fair that is not a particularly easy problem to solve!
>
>
> >
> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125
> ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/
> kstream/internals/KTableImpl.java#L493
> >
> > was unideal and makes everything a mess.
>
>
> I'm not sure i agree that it makes everything a mess, but It could have
> been done differently.
>
> The JoinType:Whatever is also not really flexible. 2 things come to my
> mind:
> >
> > 1. I don't think we should rule out config based decisions say configs
> like
> >         streams.$applicationID.joins.$joinname.conf = value
> >
>
> Is this just for config? Or are you suggesting that we could somehow "code"
> the join in a config file?
>
>
> > This can allow for tremendous changes without single API change and IMO
> it
> > was not considered enough yet.
> >
> > 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
> > example can be used to implement different join types as the user wishes.
> >
>
> Do you have an example of how this might look?
>
>
> > As Gouzhang said: stopping to break users is very important.
>
>
> Of course. We want to make it as easy as possible for people to use
> streams.
>
>
> especially with this changes + All the plans I sadly only have in my head
> > but hopefully the first link can give a glimpse.
> >
> > Thanks for preparing the examples made it way clearer to me what exactly
> > we are talking about. I would argue to go a bit slower and more carefull
> on
> > this one. At some point we need to get it right. Peeking over to the
> hadoop
> > guys with their hughe userbase. Config files really work well for them.
> >
> > Best Jan
> >
> >
> >
> >
> >
> > On 30.06.2017 09:31, Damian Guy wrote:
> > > Thanks Matthias
> > >
> > > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
> > wrote:
> > >
> > >> I am just catching up on this thread, so sorry for the long email in
> > >> advance... Also, it's to some extend a dump of thoughts and not
> always a
> > >> clear proposal. Still need to think about this in more detail. But
> maybe
> > >> it helps other to get new ideas :)
> > >>
> > >>
> > >>>> However, I don't understand your argument about putting aggregate()
> > >>>> after the withXX() -- all the calls to withXX() set optional
> > parameters
> > >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> might
> > >>>> be quite confusion for developers.
> > >>>>
> > >>>>
> > >>> I see what you are saying, but the grouped stream is effectively a
> > no-op
> > >>> until you call one of the aggregate/count/reduce etc functions. So
> the
> > >>> optional params are ones that are applicable to any of the operations
> > you
> > >>> can perform on this grouped stream. Then the final
> > >>> count()/reduce()/aggregate() call has any of the params that are
> > >>> required/specific to that function.
> > >>>
> > >> I understand your argument, but you don't share the conclusion. If we
> > >> need a "final/terminal" call, the better way might be
> > >>
> > >> .groupBy().count().withXX().build()
> > >>
> > >> (with a better name for build() though)
> > >>
> > >>
> > > The point is that all the other calls, i.e,withBlah, windowed, etc
> apply
> > > too all the aggregate functions. The terminal call being the actual
> type
> > of
> > > aggregation you want to do. I personally find this more natural than
> > > groupBy().count().withBlah().build()
> > >
> > >
> > >>> groupedStream.count(/** non windowed count**/)
> > >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> > >>
> > >> I like this. However, I don't see a reason to have windowed() and
> > >> sessionWindowed(). We should have one top-level `Windows` interface
> that
> > >> both `TimeWindows` and `SessionWindows` implement and just have a
> single
> > >> windowed() method that accepts all `Windows`. (I did not like the
> > >> separation of `SessionWindows` in the first place, and this seems to
> be
> > >> an opportunity to clean this up. It was hard to change when we
> > >> introduced session windows)
> > >>
> > > Yes - true we should look into that.
> > >
> > >
> > >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> > >> might also want to use windowBy() (instead of windowed()). Not sure
> how
> > >> important this is, but it seems to be inconsistent otherwise.
> > >>
> > >>
> > > Makes sense
> > >
> > >
> > >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
> think,
> > >> defining an inner/left/outer join is not an optional argument but a
> > >> first class concept and should have a proper representation in the API
> > >> (like the current methods join(), leftJoin, outerJoin()).
> > >>
> > >>
> > > Yep, i did originally have it as a required param and maybe that is
> what
> > we
> > > go with. It could have a default, but maybe that is confusing.
> > >
> > >
> > >
> > >> About the two join API proposals, the second one has too much boiler
> > >> plate code for my taste. Also, the actual join() operator has only one
> > >> argument what is weird to me, as in my thinking process, the main
> > >> operator call, should have one parameter per mandatory argument but
> your
> > >> proposal put the mandatory arguments into Joins.streamStreamJoin()
> call.
> > >> This is far from intuitive IMHO.
> > >>
> > >>
> > > This is the builder pattern, you only need one param as the builder has
> > > captured all of the required and optional arguments.
> > >
> > >
> > >> The first join proposal also seems to align better with the pattern
> > >> suggested for aggregations and having the same pattern for all
> operators
> > >> is important (as you stated already).
> > >>
> > >>
> > > This is why i offered two alternatives as i started out with. 1 is the
> > > builder pattern, the other is the more fluent pattern.
> > >
> > >
> > >>
> > >> Coming back to the config vs optional parameter. What about having a
> > >> method withConfig[s](...) that allow to put in the configuration?
> > >>
> > >>
> > > Sure, it is currently called withLogConfig() as that is the only thing
> > that
> > > is really config.
> > >
> > >
> > >> This also raises the question if until() is a windows property?
> > >> Actually, until() seems to be a configuration parameter and thus,
> should
> > >> not not have it's own method.
> > >>
> > >>
> > > Hmmm, i don't agree. Until is a property of the window. It is going to
> be
> > > potentially different for every window operation you do in a streams
> app.
> > >
> > >
> > >>
> > >> Browsing throw your example DSL branch, I also saw this one:
> > >>
> > >>> final KTable<Windowed<String>, Long> windowed>
> > >>   groupedStream.counting()
> > >>>                   .windowed(TimeWindows.of(10L).until(10))
> > >>>                   .table();
> > >> This is an interesting idea, and it remind my on some feedback about
> "I
> > >> wanted to count a stream, but there was no count() method -- I first
> > >> needed to figure out, that I need to group the stream first to be able
> > >> to count it. It does make sense in hindsight but was not obvious in
> the
> > >> beginning". Thus, carrying out this thought, we could also do the
> > >> following:
> > >>
> > >> stream.count().groupedBy().windowedBy().table();
> > >>
> > >> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> > >> it comes after the count()
> > >>
> > >> This would be more consistent than your proposal (that has grouping
> > >> before but windowing after count()). It might even allow us to enrich
> > >> the API with a some syntactic sugar like `stream.count().table()` to
> get
> > >> the overall count of all records (this would obviously not scale, but
> we
> > >> could support it -- if not now, maybe later).
> > >>
> > >>
> > > I guess i'd prefer
> > > stream.groupBy().windowBy().count()
> > > stream.groupBy().windowBy().reduce()
> > > stream.groupBy().count()
> > >
> > > As i said above, everything that happens before the final aggregate
> call
> > > can be applied to any of them. So it makes sense to me to do those
> things
> > > ahead of the final aggregate call.
> > >
> > >
> > >> Last about builder pattern. I am convinced that we need some
> "terminal"
> > >> operator/method that tells us when to add the processor to the
> topology.
> > >> But I don't see the need for a plain builder pattern that feels alien
> to
> > >> me (see my argument about the second join proposal). Using .stream() /
> > >> .table() as use in many examples might work. But maybe a more generic
> > >> name that we can use in all places like build() or apply() might also
> be
> > >> an option.
> > >>
> > >>
> > > Sure, a generic name might be ok.
> > >
> > >
> > >
> > >
> > >> -Matthias
> > >>
> > >>
> > >>
> > >> On 6/29/17 7:37 AM, Damian Guy wrote:
> > >>> Thanks Kyle.
> > >>>
> > >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
> winkelman.kyle@gmail.com>
> > >>> wrote:
> > >>>
> > >>>> Hi Damian,
> > >>>>
> > >>>>>>>> When trying to program in the fluent API that has been discussed
> > >> most
> > >>>> it
> > >>>>>>>> feels difficult to know when you will actually get an object you
> > can
> > >>>> reuse.
> > >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
> > legal
> > >> to
> > >>>>>>>> reuse it or does this approach expect you to call grouped each
> > time?
> > >>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
> it
> > >> as
> > >>>> you
> > >>>>>> can today.
> > >>>> You said it yourself in another post that the grouped stream is
> > >>>> effectively a no-op until a count, reduce, or aggregate. The way I
> see
> > >> it
> > >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
> > >> because
> > >>>> most of this fluent api would continue returning this (this being
> the
> > >>>> builder object currently being manipulated).
> > >>> So, if you ever store a reference to anything but KStreams and
> KTables
> > >> and
> > >>>> you use it in two different ways then its possible you make
> > conflicting
> > >>>> withXXX() calls on the same builder.
> > >>>>
> > >>>>
> > >>> No necessarily true. It could return a new instance of the builder,
> > i.e.,
> > >>> the builders being immutable. So if you held a reference to the
> builder
> > >> it
> > >>> would always be the same as it was when it was created.
> > >>>
> > >>>
> > >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
> kStream.grouped();
> > >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> > >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> > >>>>
> > >>>> I’ll admit that this shouldn’t happen but some user is going to do
> it
> > >>>> eventually…
> > >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
> > would
> > >>>> most likely be equivalent to the version withDeclaredSerdes. One
> work
> > >>>> around would be to always make copies of the config objects you are
> > >>>> building, but this approach has its own problem because now we have
> to
> > >>>> identify which configs are equivalent so we don’t create repeated
> > >>>> processors.
> > >>>>
> > >>>> The point of this long winded example is that we always have to be
> > >>>> thinking about all of the possible ways it could be misused by a
> user
> > >>>> (causing them to see hard to diagnose problems).
> > >>>>
> > >>> Exactly! That is the point of the discussion really.
> > >>>
> > >>>
> > >>>> In my attempt at a couple methods with builders I feel that I could
> > >>>> confidently say the user couldn’t really mess it up.
> > >>>>> // Count
> > >>>>> KTable<String, Long> count =
> > >>>>>
> > kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> > >>>> The kGroupedStream is reusable and if they attempted to reuse the
> > Count
> > >>>> for some reason it would throw an error message saying that a store
> > >> named
> > >>>> “my-store” already exists.
> > >>>>
> > >>>>
> > >>> Yes i agree and i think using builders is my preferred pattern.
> > >>>
> > >>> Cheers,
> > >>> Damian
> > >>>
> > >>>
> > >>>> Thanks,
> > >>>> Kyle
> > >>>>
> > >>>> From: Damian Guy
> > >>>> Sent: Thursday, June 29, 2017 3:59 AM
> > >>>> To: dev@kafka.apache.org
> > >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> > >>>>
> > >>>> Hi Kyle,
> > >>>>
> > >>>> Thanks for your input. Really appreciated.
> > >>>>
> > >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
> winkelman.kyle@gmail.com
> > >
> > >>>> wrote:
> > >>>>
> > >>>>> I like more of a builder pattern even though others have voiced
> > against
> > >>>>> it. The reason I like it is because it makes it clear to the user
> > that
> > >> a
> > >>>>> call to KGroupedStream#count will return a KTable not some
> > intermediate
> > >>>>> class that I need to undetstand.
> > >>>>>
> > >>>> Yes, that makes sense.
> > >>>>
> > >>>>
> > >>>>> When trying to program in the fluent API that has been discussed
> most
> > >> it
> > >>>>> feels difficult to know when you will actually get an object you
> can
> > >>>> reuse.
> > >>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> > to
> > >>>>> reuse it or does this approach expect you to call grouped each
> time?
> > >>>>
> > >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> > as
> > >> you
> > >>>> can today.
> > >>>>
> > >>>>
> > >>>>> This question doesn’t pop into my head at all in the builder
> pattern
> > I
> > >>>>> assume I can reuse everything.
> > >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
> > fan
> > >> of
> > >>>>> the grouped.
> > >>>>>
> > >>>>> Yes, grouped() was more for demonstration and because groupBy() and
> > >>>> groupByKey() were taken! So i'd imagine the api would actually want
> to
> > >> be
> > >>>> groupByKey(/** no required args***/).withOptionalArg() and
> > >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> > >> depends
> > >>>> on maintaining backward compatibility.
> > >>>>
> > >>>>
> > >>>>> Unfortunately, the below approach would require atleast 2 (probably
> > 3)
> > >>>>> overloads (one for returning a KTable and one for returning a
> KTable
> > >> with
> > >>>>> Windowed Key, probably would want to split windowed and
> > sessionwindowed
> > >>>> for
> > >>>>> ease of implementation) of each count, reduce, and aggregate.
> > >>>>> Obviously not exhaustive but enough for you to get the picture.
> > Count,
> > >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
> > >> builder:
> > >>>>> // Count
> > >>>>> KTable<String, Long> count =
> > >>>>>
> > groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>> // Windowed Count
> > >>>>> KTable<Windowed<String>, Long> windowedCount =
> > >>>>>
> > >>
> > groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).
> withQueryableStoreName("my-windowed-store"));
> > >>>>> // Session Count
> > >>>>> KTable<Windowed<String>, Long> sessionCount =
> > >>>>>
> > >>
> > groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).
> withQueryableStoreName("my-session-windowed-store"));
> > >>>>>
> > >>>> Above and below, i think i'd prefer it to be:
> > >>>> groupedStream.count(/** non windowed count**/)
> > >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>> // Reduce
> > >>>>> Reducer<Long> reducer;
> > >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> > >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>> // Aggregate Windowed with Custom Store
> > >>>>> Initializer<String> initializer;
> > >>>>> Aggregator<String, Long, String> aggregator;
> > >>>>> KTable<Windowed<String>, String> aggregate =
> > >>>>> groupedStream.aggregate(initializer, aggregator,
> > >>>>>
> > >>
> > Aggregate.windowed(TimeWindows.of(10L).until(10))
> .withStateStoreSupplier(stateStoreSupplier)));
> > >>>>> // Cogroup SessionWindowed
> > >>>>> KTable<String, String> cogrouped =
> > groupedStream1.cogroup(aggregator1)
> > >>>>>          .cogroup(groupedStream2, aggregator2)
> > >>>>>          .aggregate(initializer, aggregator,
> > >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> > >>>>> sessionMerger).withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> public class Count {
> > >>>>>
> > >>>>>      public static class Windowed extends Count {
> > >>>>>          private Windows windows;
> > >>>>>      }
> > >>>>>      public static class SessionWindowed extends Count {
> > >>>>>          private SessionWindows sessionWindows;
> > >>>>>      }
> > >>>>>
> > >>>>>      public static Count count();
> > >>>>>      public static Windowed windowed(Windows windows);
> > >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
> > >>>>> sessionWindows);
> > >>>>>
> > >>>>>      // All withXXX(...) methods.
> > >>>>> }
> > >>>>>
> > >>>>> public class KGroupedStream {
> > >>>>>      public KTable<K, Long> count(Count count);
> > >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
> > >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> > >> count);
> > >>>>> …
> > >>>>> }
> > >>>>>
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Kyle
> > >>>>>
> > >>>>> From: Guozhang Wang
> > >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
> > >>>>> To: dev@kafka.apache.org
> > >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> > >>>>>
> > >>>>> I played the current proposal a bit with
> > >> https://github.com/dguy/kafka/
> > >>>>> tree/dsl-experiment <
> > https://github.com/dguy/kafka/tree/dsl-experiment
> > >>> ,
> > >>>>> and here are my observations:
> > >>>>>
> > >>>>> 1. Personally I prefer
> > >>>>>
> > >>>>>      "stream.group(mapper) / stream.groupByKey()"
> > >>>>>
> > >>>>> than
> > >>>>>
> > >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
> > >>>>>
> > >>>>> Since 1) withKeyMapper is not enforced programmatically though it
> is
> > >> not
> > >>>>> "really" optional like others, 2) syntax-wise it reads more
> natural.
> > >>>>>
> > >>>>> I think it is okay to add the APIs in (
> > >>>>>
> > >>>>>
> > >>
> > https://github.com/dguy/kafka/blob/dsl-experiment/streams/
> src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> > >>>>> )
> > >>>>> in KGroupedStream.
> > >>>>>
> > >>>>>
> > >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
> > pass
> > >> in
> > >>>>> the most-inner state store supplier (e.g. then one whose get()
> return
> > >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
> > with
> > >>>>> logging / metrics / etc? I think it would be more useful to only
> > >> require
> > >>>>> users pass in the inner state store supplier while specifying
> > caching /
> > >>>>> logging through other APIs.
> > >>>>>
> > >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
> me:
> > we
> > >>>> are
> > >>>>> allowing users to call other APIs like "withQueryableName" multiple
> > >> time,
> > >>>>> but only call "withStateStoreSupplier" only once in the end. Why is
> > >> that?
> > >>>>>
> > >>>>> 3. The current DSL seems to be only for aggregations, what about
> > joins?
> > >>>>>
> > >>>>>
> > >>>>> 4. I think it is okay to keep the "withLogConfig": for the
> > >>>>> StateStoreSupplier it will still be user code specifying the
> topology
> > >> so
> > >>>> I
> > >>>>> do not see there is a big difference.
> > >>>>>
> > >>>>>
> > >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
> the
> > >>>>> windowed state store supplier to enforce typing?
> > >>>>>
> > >>>>>
> > >>>>> Below are minor ones:
> > >>>>>
> > >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> > >>>>>
> > >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> > >> matthias@confluent.io>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> I see your point about "when to add the processor to the
> topology".
> > >>>> That
> > >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
> > >>>>> topology...
> > >>>>>> I don't see any problem with having all the withXX() in KTable
> > >>>> interface
> > >>>>>> -- but this might be subjective.
> > >>>>>>
> > >>>>>>
> > >>>>>> However, I don't understand your argument about putting
> aggregate()
> > >>>>>> after the withXX() -- all the calls to withXX() set optional
> > >> parameters
> > >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> > might
> > >>>>>> be quite confusion for developers.
> > >>>>>>
> > >>>>>>
> > >>>>>> -Matthias
> > >>>>>>
> > >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> > >>>>>>>> I also think that mixing optional parameters with configs is a
> bad
> > >>>>> idea.
> > >>>>>>>> Have not proposal for this atm but just wanted to mention it.
> Hope
> > >>>> to
> > >>>>>>>> find some time to come up with something.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> Yes, i don't like the mix of config either. But the only real
> > config
> > >>>>> here
> > >>>>>>> is the logging config - which we don't really need as it can
> > already
> > >>>> be
> > >>>>>>> done via a custom StateStoreSupplier.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> What I don't like in the current proposal is the
> > >>>>>>>> .grouped().withKeyMapper() -- the current solution with
> > >>>> .groupBy(...)
> > >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
> > >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
> > find
> > >>>>>>>> some better names).
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> it could be groupByKey(), groupBy() or something different bt
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> The proposed pattern "chains" grouping and aggregation too close
> > >>>>>>>> together. I would rather separate both more than less, ie, do
> into
> > >>>> the
> > >>>>>>>> opposite direction.
> > >>>>>>>>
> > >>>>>>>> I am also wondering, if we could so something more "fluent". The
> > >>>>> initial
> > >>>>>>>> proposal was like:
> > >>>>>>>>
> > >>>>>>>>>> groupedStream.count()
> > >>>>>>>>>>     .withStoreName("name")
> > >>>>>>>>>>     .withCachingEnabled(false)
> > >>>>>>>>>>     .withLoggingEnabled(config)
> > >>>>>>>>>>     .table()
> > >>>>>>>> The .table() statement in the end was kinda alien.
> > >>>>>>>>
> > >>>>>>> I agree, but then all of the withXXX methods need to be on KTable
> > >>>> which
> > >>>>>> is
> > >>>>>>> worse in my opinion. You also need something that is going to
> > "build"
> > >>>>> the
> > >>>>>>> internal processors and add them to the topology.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> The current proposal put the count() into the end -- ie, the
> > >>>> optional
> > >>>>>>>> parameter for count() have to specified on the .grouped() call
> --
> > >>>> this
> > >>>>>>>> does not seems to be the best way either.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> I actually prefer this method as you are building a grouped
> stream
> > >>>> that
> > >>>>>> you
> > >>>>>>> will aggregate. So
> > >>>> table.grouped(...).withOptionalStuff().aggregate(..)
> > >>>>>> etc
> > >>>>>>> seems natural to me.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> I did not think this through in detail, but can't we just do the
> > >>>>> initial
> > >>>>>>>> proposal with the .table() ?
> > >>>>>>>>
> > >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> > >>>>>>>>
> > >>>>>>>> Each .withXXX(...) return the current KTable and all the
> > .withXXX()
> > >>>>> are
> > >>>>>>>> just added to the KTable interface. Or do I miss anything why
> this
> > >>>>> wont'
> > >>>>>>>> work or any obvious disadvantage?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> See above.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> -Matthias
> > >>>>>>>>
> > >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> > >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> > >>>> fluent
> > >>>>>>>>> approach, but i think it is slightly nicer.
> > >>>>>>>>> I agree with some of what Eno said about mixing configy stuff
> in
> > >>>> the
> > >>>>>> DSL,
> > >>>>>>>>> but i think that enabling caching and enabling logging are
> things
> > >>>>> that
> > >>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> > >>>> (even
> > >>>>>>>>> though it is below) as this is actually config and we already
> > have
> > >>>> a
> > >>>>>> way
> > >>>>>>>> of
> > >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
> the
> > >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
> > that
> > >>>>> is
> > >>>>>> a
> > >>>>>>>>> bit of a tedious process for someone that just wants to use the
> > >>>>> default
> > >>>>>>>>> storage engine, but not have caching enabled.
> > >>>>>>>>>
> > >>>>>>>>> There is also an orthogonal concern that Guozhang alluded
> to....
> > If
> > >>>>> you
> > >>>>>>>>> want to plug in a custom storage engine and you want it to be
> > >>>> logged
> > >>>>>> etc,
> > >>>>>>>>> you would currently need to implement that yourself. Ideally we
> > can
> > >>>>>>>> provide
> > >>>>>>>>> a way where we will wrap the custom store with logging,
> metrics,
> > >>>>> etc. I
> > >>>>>>>>> need to think about where this fits, it is probably more
> > >>>> appropriate
> > >>>>> on
> > >>>>>>>> the
> > >>>>>>>>> Stores API.
> > >>>>>>>>>
> > >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > >>>>>>>>> // count with mapped key
> > >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
> > >>>>>>>>>          .withKeyMapper(keyMapper)
> > >>>>>>>>>          .withKeySerde(Serdes.Long())
> > >>>>>>>>>          .withValueSerde(Serdes.String())
> > >>>>>>>>>          .withQueryableName("my-store")
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // windowed count
> > >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> > >>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-window-store")
> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // windowed reduce
> > >>>>>>>>> final Reducer<String> windowedReducer = null;
> > >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> > >>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-window-store")
> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> > >>>>>>>>>          .reduce(windowedReducer);
> > >>>>>>>>>
> > >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
> > >>>>>>>>> final Initializer<Long> init = null;
> > >>>>>>>>>
> > >>>>>>>>> // aggregate
> > >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-aggregate-store")
> > >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
> > >>>>>>>>>
> > >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > >>>>>> stateStoreSupplier
> > >>>>>>>> = null;
> > >>>>>>>>> // aggregate with custom store
> > >>>>>>>>> final KTable<String, Long> aggWithCustomStore =
> stream.grouped()
> > >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
> > >>>>>>>>>          .aggregate(aggregator, init);
> > >>>>>>>>>
> > >>>>>>>>> // disable caching
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("name")
> > >>>>>>>>>          .withCachingEnabled(false)
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // disable logging
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("q")
> > >>>>>>>>>          .withLoggingEnabled(false)
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // override log config
> > >>>>>>>>> final Reducer<String> reducer = null;
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withLogConfig(Collections.
> singletonMap("segment.size",
> > >>>>> "10"))
> > >>>>>>>>>          .reduce(reducer);
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> If anyone wants to play around with this you can find the code
> > >>>> here:
> > >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> > >>>>>>>>>
> > >>>>>>>>> Note: It won't actually work as most of the methods just return
> > >>>> null.
> > >>>>>>>>> Thanks,
> > >>>>>>>>> Damian
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> > >>>> wrote:
> > >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
> both
> > >>>> are
> > >>>>>>>> better
> > >>>>>>>>>> than overload abuse.
> > >>>>>>>>>>
> > >>>>>>>>>> The fluent API approach reads better, no mention of builder or
> > >>>> build
> > >>>>>>>>>> anywhere. The main downside is that the method signatures are
> a
> > >>>>> little
> > >>>>>>>> less
> > >>>>>>>>>> clear. By reading the method signature, one doesn't
> necessarily
> > >>>>> knows
> > >>>>>>>> what
> > >>>>>>>>>> it returns. Also, one needs to figure out the special method
> > >>>>>> (`table()`
> > >>>>>>>> in
> > >>>>>>>>>> this case) that gives you what you actually care about
> (`KTable`
> > >>>> in
> > >>>>>> this
> > >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
> > >>>>>>>> comparison.
> > >>>>>>>>>> The builder approach avoids the issues mentioned above, but it
> > >>>>> doesn't
> > >>>>>>>> read
> > >>>>>>>>>> as well.
> > >>>>>>>>>>
> > >>>>>>>>>> Ismael
> > >>>>>>>>>>
> > >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
> > damian.guy@gmail.com
> > >>>>>>>> wrote:
> > >>>>>>>>>>> Hi,
> > >>>>>>>>>>>
> > >>>>>>>>>>> I'd like to get a discussion going around some of the API
> > choices
> > >>>>>> we've
> > >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
> > >>>>>> operations
> > >>>>>>>>>>> (though this could expand).
> > >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
> API,
> > >>>> i.e,
> > >>>>>>>> there
> > >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> > >>>> noisy
> > >>>>>> and
> > >>>>>>>> i
> > >>>>>>>>>>> feel it is only going to get worse as we add more optional
> > >>>> params.
> > >>>>> In
> > >>>>>>>>>>> particular we've had some requests to be able to turn caching
> > >>>> off,
> > >>>>> or
> > >>>>>>>>>>> change log configs,  on a per operator basis (note this can
> be
> > >>>> done
> > >>>>>> now
> > >>>>>>>>>> if
> > >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> > >>>>> cumbersome).
> > >>>>>>>>>>> So this is a bit of an open question. How can we change the
> DSL
> > >>>>>>>> overloads
> > >>>>>>>>>>> so that it flows, is simple to use and understand, and is
> > easily
> > >>>>>>>> extended
> > >>>>>>>>>>> in the future?
> > >>>>>>>>>>>
> > >>>>>>>>>>> One option would be to use a fluent API approach for
> providing
> > >>>> the
> > >>>>>>>>>> optional
> > >>>>>>>>>>> params, so something like this:
> > >>>>>>>>>>>
> > >>>>>>>>>>> groupedStream.count()
> > >>>>>>>>>>>     .withStoreName("name")
> > >>>>>>>>>>>     .withCachingEnabled(false)
> > >>>>>>>>>>>     .withLoggingEnabled(config)
> > >>>>>>>>>>>     .table()
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Another option would be to provide a Builder to the count
> > method,
> > >>>>> so
> > >>>>>> it
> > >>>>>>>>>>> would look something like this:
> > >>>>>>>>>>> groupedStream.count(new
> > >>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>>>>>>>>>>
> > >>>>>>>>>>> Another option is to say: Hey we don't need this, what are
> you
> > on
> > >>>>>>>> about!
> > >>>>>>>>>>> The above has focussed on state store related overloads, but
> > the
> > >>>>> same
> > >>>>>>>>>> ideas
> > >>>>>>>>>>> could  be applied to joins etc, where we presently have many
> > join
> > >>>>>>>> methods
> > >>>>>>>>>>> and many overloads.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>> Damian
> > >>>>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>> --
> > >>>>> -- Guozhang
> > >>>>>
> > >>>>>
> > >>>>
> > >>
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Damian / Kyle,

I think I agree with you guys about the pros / cons of using the builder
pattern v.s. using some "secondary classes". And I'm thinking if we can
take a "mid" manner between these two. I spent some time with a slight
different approach from Damian's current proposal:

https://github.com/guozhangwang/kafka/blob/dsl-refactor/streams/src/main/java/org/apache/kafka/streams/RefactoredAPIs.java

The key idea is to tolerate the final "table()" or "stream()" function to
"upgrade" from the secondary classes to the first citizen classes, while
having all the specs inside this function. Also this proposal includes some
other refactoring that people have been discussed about for the builder to
reduce the overloaded functions as well. WDYT?


Guozhang


On Tue, Jul 4, 2017 at 1:40 AM, Damian Guy <da...@gmail.com> wrote:

> Hi Jan,
>
> Thanks very much for the input.
>
> On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com> wrote:
>
> > Hi Damian,
> >
> > I do see your point of something needs to change. But I fully agree with
> > Gouzhang when he says.
> > ---
> >
> > But since this is a incompatibility change, and we are going to remove
> the
> > compatibility annotations soon it means we only have one chance and we
> > really have to make it right.
> > ----
> >
> >
> I think we all agree on this one! Hence the discussion.
>
>
> > I fear all suggestions do not go far enough to become something that will
> > carry on for very much longer.
> > I am currently working on KAFKA-3705 and try to find the most easy way
> for
> > the user to give me all the required functionality. The easiest
> interface I
> > could come up so far can be looked at here.
> >
> >
> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125
> ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/
> kstream/internals/KTableImpl.java#L622
> >
> >
> And its already horribly complicated. I am currently unable to find the
> > right abstraction level to have everything falling into place naturally.
> To
> > be honest I already think introducing
> >
> >
> To be fair that is not a particularly easy problem to solve!
>
>
> >
> > https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125
> ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/
> kstream/internals/KTableImpl.java#L493
> >
> > was unideal and makes everything a mess.
>
>
> I'm not sure i agree that it makes everything a mess, but It could have
> been done differently.
>
> The JoinType:Whatever is also not really flexible. 2 things come to my
> mind:
> >
> > 1. I don't think we should rule out config based decisions say configs
> like
> >         streams.$applicationID.joins.$joinname.conf = value
> >
>
> Is this just for config? Or are you suggesting that we could somehow "code"
> the join in a config file?
>
>
> > This can allow for tremendous changes without single API change and IMO
> it
> > was not considered enough yet.
> >
> > 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
> > example can be used to implement different join types as the user wishes.
> >
>
> Do you have an example of how this might look?
>
>
> > As Gouzhang said: stopping to break users is very important.
>
>
> Of course. We want to make it as easy as possible for people to use
> streams.
>
>
> especially with this changes + All the plans I sadly only have in my head
> > but hopefully the first link can give a glimpse.
> >
> > Thanks for preparing the examples made it way clearer to me what exactly
> > we are talking about. I would argue to go a bit slower and more carefull
> on
> > this one. At some point we need to get it right. Peeking over to the
> hadoop
> > guys with their hughe userbase. Config files really work well for them.
> >
> > Best Jan
> >
> >
> >
> >
> >
> > On 30.06.2017 09:31, Damian Guy wrote:
> > > Thanks Matthias
> > >
> > > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
> > wrote:
> > >
> > >> I am just catching up on this thread, so sorry for the long email in
> > >> advance... Also, it's to some extend a dump of thoughts and not
> always a
> > >> clear proposal. Still need to think about this in more detail. But
> maybe
> > >> it helps other to get new ideas :)
> > >>
> > >>
> > >>>> However, I don't understand your argument about putting aggregate()
> > >>>> after the withXX() -- all the calls to withXX() set optional
> > parameters
> > >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> might
> > >>>> be quite confusion for developers.
> > >>>>
> > >>>>
> > >>> I see what you are saying, but the grouped stream is effectively a
> > no-op
> > >>> until you call one of the aggregate/count/reduce etc functions. So
> the
> > >>> optional params are ones that are applicable to any of the operations
> > you
> > >>> can perform on this grouped stream. Then the final
> > >>> count()/reduce()/aggregate() call has any of the params that are
> > >>> required/specific to that function.
> > >>>
> > >> I understand your argument, but you don't share the conclusion. If we
> > >> need a "final/terminal" call, the better way might be
> > >>
> > >> .groupBy().count().withXX().build()
> > >>
> > >> (with a better name for build() though)
> > >>
> > >>
> > > The point is that all the other calls, i.e,withBlah, windowed, etc
> apply
> > > too all the aggregate functions. The terminal call being the actual
> type
> > of
> > > aggregation you want to do. I personally find this more natural than
> > > groupBy().count().withBlah().build()
> > >
> > >
> > >>> groupedStream.count(/** non windowed count**/)
> > >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> > >>
> > >> I like this. However, I don't see a reason to have windowed() and
> > >> sessionWindowed(). We should have one top-level `Windows` interface
> that
> > >> both `TimeWindows` and `SessionWindows` implement and just have a
> single
> > >> windowed() method that accepts all `Windows`. (I did not like the
> > >> separation of `SessionWindows` in the first place, and this seems to
> be
> > >> an opportunity to clean this up. It was hard to change when we
> > >> introduced session windows)
> > >>
> > > Yes - true we should look into that.
> > >
> > >
> > >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> > >> might also want to use windowBy() (instead of windowed()). Not sure
> how
> > >> important this is, but it seems to be inconsistent otherwise.
> > >>
> > >>
> > > Makes sense
> > >
> > >
> > >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I
> think,
> > >> defining an inner/left/outer join is not an optional argument but a
> > >> first class concept and should have a proper representation in the API
> > >> (like the current methods join(), leftJoin, outerJoin()).
> > >>
> > >>
> > > Yep, i did originally have it as a required param and maybe that is
> what
> > we
> > > go with. It could have a default, but maybe that is confusing.
> > >
> > >
> > >
> > >> About the two join API proposals, the second one has too much boiler
> > >> plate code for my taste. Also, the actual join() operator has only one
> > >> argument what is weird to me, as in my thinking process, the main
> > >> operator call, should have one parameter per mandatory argument but
> your
> > >> proposal put the mandatory arguments into Joins.streamStreamJoin()
> call.
> > >> This is far from intuitive IMHO.
> > >>
> > >>
> > > This is the builder pattern, you only need one param as the builder has
> > > captured all of the required and optional arguments.
> > >
> > >
> > >> The first join proposal also seems to align better with the pattern
> > >> suggested for aggregations and having the same pattern for all
> operators
> > >> is important (as you stated already).
> > >>
> > >>
> > > This is why i offered two alternatives as i started out with. 1 is the
> > > builder pattern, the other is the more fluent pattern.
> > >
> > >
> > >>
> > >> Coming back to the config vs optional parameter. What about having a
> > >> method withConfig[s](...) that allow to put in the configuration?
> > >>
> > >>
> > > Sure, it is currently called withLogConfig() as that is the only thing
> > that
> > > is really config.
> > >
> > >
> > >> This also raises the question if until() is a windows property?
> > >> Actually, until() seems to be a configuration parameter and thus,
> should
> > >> not not have it's own method.
> > >>
> > >>
> > > Hmmm, i don't agree. Until is a property of the window. It is going to
> be
> > > potentially different for every window operation you do in a streams
> app.
> > >
> > >
> > >>
> > >> Browsing throw your example DSL branch, I also saw this one:
> > >>
> > >>> final KTable<Windowed<String>, Long> windowed>
> > >>   groupedStream.counting()
> > >>>                   .windowed(TimeWindows.of(10L).until(10))
> > >>>                   .table();
> > >> This is an interesting idea, and it remind my on some feedback about
> "I
> > >> wanted to count a stream, but there was no count() method -- I first
> > >> needed to figure out, that I need to group the stream first to be able
> > >> to count it. It does make sense in hindsight but was not obvious in
> the
> > >> beginning". Thus, carrying out this thought, we could also do the
> > >> following:
> > >>
> > >> stream.count().groupedBy().windowedBy().table();
> > >>
> > >> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> > >> it comes after the count()
> > >>
> > >> This would be more consistent than your proposal (that has grouping
> > >> before but windowing after count()). It might even allow us to enrich
> > >> the API with a some syntactic sugar like `stream.count().table()` to
> get
> > >> the overall count of all records (this would obviously not scale, but
> we
> > >> could support it -- if not now, maybe later).
> > >>
> > >>
> > > I guess i'd prefer
> > > stream.groupBy().windowBy().count()
> > > stream.groupBy().windowBy().reduce()
> > > stream.groupBy().count()
> > >
> > > As i said above, everything that happens before the final aggregate
> call
> > > can be applied to any of them. So it makes sense to me to do those
> things
> > > ahead of the final aggregate call.
> > >
> > >
> > >> Last about builder pattern. I am convinced that we need some
> "terminal"
> > >> operator/method that tells us when to add the processor to the
> topology.
> > >> But I don't see the need for a plain builder pattern that feels alien
> to
> > >> me (see my argument about the second join proposal). Using .stream() /
> > >> .table() as use in many examples might work. But maybe a more generic
> > >> name that we can use in all places like build() or apply() might also
> be
> > >> an option.
> > >>
> > >>
> > > Sure, a generic name might be ok.
> > >
> > >
> > >
> > >
> > >> -Matthias
> > >>
> > >>
> > >>
> > >> On 6/29/17 7:37 AM, Damian Guy wrote:
> > >>> Thanks Kyle.
> > >>>
> > >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <
> winkelman.kyle@gmail.com>
> > >>> wrote:
> > >>>
> > >>>> Hi Damian,
> > >>>>
> > >>>>>>>> When trying to program in the fluent API that has been discussed
> > >> most
> > >>>> it
> > >>>>>>>> feels difficult to know when you will actually get an object you
> > can
> > >>>> reuse.
> > >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
> > legal
> > >> to
> > >>>>>>>> reuse it or does this approach expect you to call grouped each
> > time?
> > >>>>>> I'd anticipate that once you have a KGroupedStream you can re-use
> it
> > >> as
> > >>>> you
> > >>>>>> can today.
> > >>>> You said it yourself in another post that the grouped stream is
> > >>>> effectively a no-op until a count, reduce, or aggregate. The way I
> see
> > >> it
> > >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
> > >> because
> > >>>> most of this fluent api would continue returning this (this being
> the
> > >>>> builder object currently being manipulated).
> > >>> So, if you ever store a reference to anything but KStreams and
> KTables
> > >> and
> > >>>> you use it in two different ways then its possible you make
> > conflicting
> > >>>> withXXX() calls on the same builder.
> > >>>>
> > >>>>
> > >>> No necessarily true. It could return a new instance of the builder,
> > i.e.,
> > >>> the builders being immutable. So if you held a reference to the
> builder
> > >> it
> > >>> would always be the same as it was when it was created.
> > >>>
> > >>>
> > >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes =
> kStream.grouped();
> > >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> > >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> > >>>>
> > >>>> I’ll admit that this shouldn’t happen but some user is going to do
> it
> > >>>> eventually…
> > >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
> > would
> > >>>> most likely be equivalent to the version withDeclaredSerdes. One
> work
> > >>>> around would be to always make copies of the config objects you are
> > >>>> building, but this approach has its own problem because now we have
> to
> > >>>> identify which configs are equivalent so we don’t create repeated
> > >>>> processors.
> > >>>>
> > >>>> The point of this long winded example is that we always have to be
> > >>>> thinking about all of the possible ways it could be misused by a
> user
> > >>>> (causing them to see hard to diagnose problems).
> > >>>>
> > >>> Exactly! That is the point of the discussion really.
> > >>>
> > >>>
> > >>>> In my attempt at a couple methods with builders I feel that I could
> > >>>> confidently say the user couldn’t really mess it up.
> > >>>>> // Count
> > >>>>> KTable<String, Long> count =
> > >>>>>
> > kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> > >>>> The kGroupedStream is reusable and if they attempted to reuse the
> > Count
> > >>>> for some reason it would throw an error message saying that a store
> > >> named
> > >>>> “my-store” already exists.
> > >>>>
> > >>>>
> > >>> Yes i agree and i think using builders is my preferred pattern.
> > >>>
> > >>> Cheers,
> > >>> Damian
> > >>>
> > >>>
> > >>>> Thanks,
> > >>>> Kyle
> > >>>>
> > >>>> From: Damian Guy
> > >>>> Sent: Thursday, June 29, 2017 3:59 AM
> > >>>> To: dev@kafka.apache.org
> > >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> > >>>>
> > >>>> Hi Kyle,
> > >>>>
> > >>>> Thanks for your input. Really appreciated.
> > >>>>
> > >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <
> winkelman.kyle@gmail.com
> > >
> > >>>> wrote:
> > >>>>
> > >>>>> I like more of a builder pattern even though others have voiced
> > against
> > >>>>> it. The reason I like it is because it makes it clear to the user
> > that
> > >> a
> > >>>>> call to KGroupedStream#count will return a KTable not some
> > intermediate
> > >>>>> class that I need to undetstand.
> > >>>>>
> > >>>> Yes, that makes sense.
> > >>>>
> > >>>>
> > >>>>> When trying to program in the fluent API that has been discussed
> most
> > >> it
> > >>>>> feels difficult to know when you will actually get an object you
> can
> > >>>> reuse.
> > >>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> > to
> > >>>>> reuse it or does this approach expect you to call grouped each
> time?
> > >>>>
> > >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> > as
> > >> you
> > >>>> can today.
> > >>>>
> > >>>>
> > >>>>> This question doesn’t pop into my head at all in the builder
> pattern
> > I
> > >>>>> assume I can reuse everything.
> > >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
> > fan
> > >> of
> > >>>>> the grouped.
> > >>>>>
> > >>>>> Yes, grouped() was more for demonstration and because groupBy() and
> > >>>> groupByKey() were taken! So i'd imagine the api would actually want
> to
> > >> be
> > >>>> groupByKey(/** no required args***/).withOptionalArg() and
> > >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> > >> depends
> > >>>> on maintaining backward compatibility.
> > >>>>
> > >>>>
> > >>>>> Unfortunately, the below approach would require atleast 2 (probably
> > 3)
> > >>>>> overloads (one for returning a KTable and one for returning a
> KTable
> > >> with
> > >>>>> Windowed Key, probably would want to split windowed and
> > sessionwindowed
> > >>>> for
> > >>>>> ease of implementation) of each count, reduce, and aggregate.
> > >>>>> Obviously not exhaustive but enough for you to get the picture.
> > Count,
> > >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
> > >> builder:
> > >>>>> // Count
> > >>>>> KTable<String, Long> count =
> > >>>>>
> > groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>> // Windowed Count
> > >>>>> KTable<Windowed<String>, Long> windowedCount =
> > >>>>>
> > >>
> > groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).
> withQueryableStoreName("my-windowed-store"));
> > >>>>> // Session Count
> > >>>>> KTable<Windowed<String>, Long> sessionCount =
> > >>>>>
> > >>
> > groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).
> withQueryableStoreName("my-session-windowed-store"));
> > >>>>>
> > >>>> Above and below, i think i'd prefer it to be:
> > >>>> groupedStream.count(/** non windowed count**/)
> > >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> > >>>>
> > >>>>
> > >>>>
> > >>>>
> > >>>>> // Reduce
> > >>>>> Reducer<Long> reducer;
> > >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> > >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>> // Aggregate Windowed with Custom Store
> > >>>>> Initializer<String> initializer;
> > >>>>> Aggregator<String, Long, String> aggregator;
> > >>>>> KTable<Windowed<String>, String> aggregate =
> > >>>>> groupedStream.aggregate(initializer, aggregator,
> > >>>>>
> > >>
> > Aggregate.windowed(TimeWindows.of(10L).until(10))
> .withStateStoreSupplier(stateStoreSupplier)));
> > >>>>> // Cogroup SessionWindowed
> > >>>>> KTable<String, String> cogrouped =
> > groupedStream1.cogroup(aggregator1)
> > >>>>>          .cogroup(groupedStream2, aggregator2)
> > >>>>>          .aggregate(initializer, aggregator,
> > >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> > >>>>> sessionMerger).withQueryableStoreName("my-store"));
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> public class Count {
> > >>>>>
> > >>>>>      public static class Windowed extends Count {
> > >>>>>          private Windows windows;
> > >>>>>      }
> > >>>>>      public static class SessionWindowed extends Count {
> > >>>>>          private SessionWindows sessionWindows;
> > >>>>>      }
> > >>>>>
> > >>>>>      public static Count count();
> > >>>>>      public static Windowed windowed(Windows windows);
> > >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
> > >>>>> sessionWindows);
> > >>>>>
> > >>>>>      // All withXXX(...) methods.
> > >>>>> }
> > >>>>>
> > >>>>> public class KGroupedStream {
> > >>>>>      public KTable<K, Long> count(Count count);
> > >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
> > >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> > >> count);
> > >>>>> …
> > >>>>> }
> > >>>>>
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Kyle
> > >>>>>
> > >>>>> From: Guozhang Wang
> > >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
> > >>>>> To: dev@kafka.apache.org
> > >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> > >>>>>
> > >>>>> I played the current proposal a bit with
> > >> https://github.com/dguy/kafka/
> > >>>>> tree/dsl-experiment <
> > https://github.com/dguy/kafka/tree/dsl-experiment
> > >>> ,
> > >>>>> and here are my observations:
> > >>>>>
> > >>>>> 1. Personally I prefer
> > >>>>>
> > >>>>>      "stream.group(mapper) / stream.groupByKey()"
> > >>>>>
> > >>>>> than
> > >>>>>
> > >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
> > >>>>>
> > >>>>> Since 1) withKeyMapper is not enforced programmatically though it
> is
> > >> not
> > >>>>> "really" optional like others, 2) syntax-wise it reads more
> natural.
> > >>>>>
> > >>>>> I think it is okay to add the APIs in (
> > >>>>>
> > >>>>>
> > >>
> > https://github.com/dguy/kafka/blob/dsl-experiment/streams/
> src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> > >>>>> )
> > >>>>> in KGroupedStream.
> > >>>>>
> > >>>>>
> > >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
> > pass
> > >> in
> > >>>>> the most-inner state store supplier (e.g. then one whose get()
> return
> > >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
> > with
> > >>>>> logging / metrics / etc? I think it would be more useful to only
> > >> require
> > >>>>> users pass in the inner state store supplier while specifying
> > caching /
> > >>>>> logging through other APIs.
> > >>>>>
> > >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to
> me:
> > we
> > >>>> are
> > >>>>> allowing users to call other APIs like "withQueryableName" multiple
> > >> time,
> > >>>>> but only call "withStateStoreSupplier" only once in the end. Why is
> > >> that?
> > >>>>>
> > >>>>> 3. The current DSL seems to be only for aggregations, what about
> > joins?
> > >>>>>
> > >>>>>
> > >>>>> 4. I think it is okay to keep the "withLogConfig": for the
> > >>>>> StateStoreSupplier it will still be user code specifying the
> topology
> > >> so
> > >>>> I
> > >>>>> do not see there is a big difference.
> > >>>>>
> > >>>>>
> > >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take
> the
> > >>>>> windowed state store supplier to enforce typing?
> > >>>>>
> > >>>>>
> > >>>>> Below are minor ones:
> > >>>>>
> > >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> > >>>>>
> > >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> > >> matthias@confluent.io>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> I see your point about "when to add the processor to the
> topology".
> > >>>> That
> > >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
> > >>>>> topology...
> > >>>>>> I don't see any problem with having all the withXX() in KTable
> > >>>> interface
> > >>>>>> -- but this might be subjective.
> > >>>>>>
> > >>>>>>
> > >>>>>> However, I don't understand your argument about putting
> aggregate()
> > >>>>>> after the withXX() -- all the calls to withXX() set optional
> > >> parameters
> > >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> > might
> > >>>>>> be quite confusion for developers.
> > >>>>>>
> > >>>>>>
> > >>>>>> -Matthias
> > >>>>>>
> > >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> > >>>>>>>> I also think that mixing optional parameters with configs is a
> bad
> > >>>>> idea.
> > >>>>>>>> Have not proposal for this atm but just wanted to mention it.
> Hope
> > >>>> to
> > >>>>>>>> find some time to come up with something.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> Yes, i don't like the mix of config either. But the only real
> > config
> > >>>>> here
> > >>>>>>> is the logging config - which we don't really need as it can
> > already
> > >>>> be
> > >>>>>>> done via a custom StateStoreSupplier.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> What I don't like in the current proposal is the
> > >>>>>>>> .grouped().withKeyMapper() -- the current solution with
> > >>>> .groupBy(...)
> > >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
> > >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
> > find
> > >>>>>>>> some better names).
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> it could be groupByKey(), groupBy() or something different bt
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> The proposed pattern "chains" grouping and aggregation too close
> > >>>>>>>> together. I would rather separate both more than less, ie, do
> into
> > >>>> the
> > >>>>>>>> opposite direction.
> > >>>>>>>>
> > >>>>>>>> I am also wondering, if we could so something more "fluent". The
> > >>>>> initial
> > >>>>>>>> proposal was like:
> > >>>>>>>>
> > >>>>>>>>>> groupedStream.count()
> > >>>>>>>>>>     .withStoreName("name")
> > >>>>>>>>>>     .withCachingEnabled(false)
> > >>>>>>>>>>     .withLoggingEnabled(config)
> > >>>>>>>>>>     .table()
> > >>>>>>>> The .table() statement in the end was kinda alien.
> > >>>>>>>>
> > >>>>>>> I agree, but then all of the withXXX methods need to be on KTable
> > >>>> which
> > >>>>>> is
> > >>>>>>> worse in my opinion. You also need something that is going to
> > "build"
> > >>>>> the
> > >>>>>>> internal processors and add them to the topology.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> The current proposal put the count() into the end -- ie, the
> > >>>> optional
> > >>>>>>>> parameter for count() have to specified on the .grouped() call
> --
> > >>>> this
> > >>>>>>>> does not seems to be the best way either.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> I actually prefer this method as you are building a grouped
> stream
> > >>>> that
> > >>>>>> you
> > >>>>>>> will aggregate. So
> > >>>> table.grouped(...).withOptionalStuff().aggregate(..)
> > >>>>>> etc
> > >>>>>>> seems natural to me.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> I did not think this through in detail, but can't we just do the
> > >>>>> initial
> > >>>>>>>> proposal with the .table() ?
> > >>>>>>>>
> > >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> > >>>>>>>>
> > >>>>>>>> Each .withXXX(...) return the current KTable and all the
> > .withXXX()
> > >>>>> are
> > >>>>>>>> just added to the KTable interface. Or do I miss anything why
> this
> > >>>>> wont'
> > >>>>>>>> work or any obvious disadvantage?
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>> See above.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>> -Matthias
> > >>>>>>>>
> > >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> > >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> > >>>> fluent
> > >>>>>>>>> approach, but i think it is slightly nicer.
> > >>>>>>>>> I agree with some of what Eno said about mixing configy stuff
> in
> > >>>> the
> > >>>>>> DSL,
> > >>>>>>>>> but i think that enabling caching and enabling logging are
> things
> > >>>>> that
> > >>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> > >>>> (even
> > >>>>>>>>> though it is below) as this is actually config and we already
> > have
> > >>>> a
> > >>>>>> way
> > >>>>>>>> of
> > >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use
> the
> > >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
> > that
> > >>>>> is
> > >>>>>> a
> > >>>>>>>>> bit of a tedious process for someone that just wants to use the
> > >>>>> default
> > >>>>>>>>> storage engine, but not have caching enabled.
> > >>>>>>>>>
> > >>>>>>>>> There is also an orthogonal concern that Guozhang alluded
> to....
> > If
> > >>>>> you
> > >>>>>>>>> want to plug in a custom storage engine and you want it to be
> > >>>> logged
> > >>>>>> etc,
> > >>>>>>>>> you would currently need to implement that yourself. Ideally we
> > can
> > >>>>>>>> provide
> > >>>>>>>>> a way where we will wrap the custom store with logging,
> metrics,
> > >>>>> etc. I
> > >>>>>>>>> need to think about where this fits, it is probably more
> > >>>> appropriate
> > >>>>> on
> > >>>>>>>> the
> > >>>>>>>>> Stores API.
> > >>>>>>>>>
> > >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > >>>>>>>>> // count with mapped key
> > >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
> > >>>>>>>>>          .withKeyMapper(keyMapper)
> > >>>>>>>>>          .withKeySerde(Serdes.Long())
> > >>>>>>>>>          .withValueSerde(Serdes.String())
> > >>>>>>>>>          .withQueryableName("my-store")
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // windowed count
> > >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> > >>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-window-store")
> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // windowed reduce
> > >>>>>>>>> final Reducer<String> windowedReducer = null;
> > >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> > >>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-window-store")
> > >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> > >>>>>>>>>          .reduce(windowedReducer);
> > >>>>>>>>>
> > >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
> > >>>>>>>>> final Initializer<Long> init = null;
> > >>>>>>>>>
> > >>>>>>>>> // aggregate
> > >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> > >>>>>>>>>          .withQueryableName("my-aggregate-store")
> > >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
> > >>>>>>>>>
> > >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > >>>>>> stateStoreSupplier
> > >>>>>>>> = null;
> > >>>>>>>>> // aggregate with custom store
> > >>>>>>>>> final KTable<String, Long> aggWithCustomStore =
> stream.grouped()
> > >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
> > >>>>>>>>>          .aggregate(aggregator, init);
> > >>>>>>>>>
> > >>>>>>>>> // disable caching
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("name")
> > >>>>>>>>>          .withCachingEnabled(false)
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // disable logging
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withQueryableName("q")
> > >>>>>>>>>          .withLoggingEnabled(false)
> > >>>>>>>>>          .count();
> > >>>>>>>>>
> > >>>>>>>>> // override log config
> > >>>>>>>>> final Reducer<String> reducer = null;
> > >>>>>>>>> stream.grouped()
> > >>>>>>>>>          .withLogConfig(Collections.
> singletonMap("segment.size",
> > >>>>> "10"))
> > >>>>>>>>>          .reduce(reducer);
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> If anyone wants to play around with this you can find the code
> > >>>> here:
> > >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> > >>>>>>>>>
> > >>>>>>>>> Note: It won't actually work as most of the methods just return
> > >>>> null.
> > >>>>>>>>> Thanks,
> > >>>>>>>>> Damian
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> > >>>> wrote:
> > >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And
> both
> > >>>> are
> > >>>>>>>> better
> > >>>>>>>>>> than overload abuse.
> > >>>>>>>>>>
> > >>>>>>>>>> The fluent API approach reads better, no mention of builder or
> > >>>> build
> > >>>>>>>>>> anywhere. The main downside is that the method signatures are
> a
> > >>>>> little
> > >>>>>>>> less
> > >>>>>>>>>> clear. By reading the method signature, one doesn't
> necessarily
> > >>>>> knows
> > >>>>>>>> what
> > >>>>>>>>>> it returns. Also, one needs to figure out the special method
> > >>>>>> (`table()`
> > >>>>>>>> in
> > >>>>>>>>>> this case) that gives you what you actually care about
> (`KTable`
> > >>>> in
> > >>>>>> this
> > >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
> > >>>>>>>> comparison.
> > >>>>>>>>>> The builder approach avoids the issues mentioned above, but it
> > >>>>> doesn't
> > >>>>>>>> read
> > >>>>>>>>>> as well.
> > >>>>>>>>>>
> > >>>>>>>>>> Ismael
> > >>>>>>>>>>
> > >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
> > damian.guy@gmail.com
> > >>>>>>>> wrote:
> > >>>>>>>>>>> Hi,
> > >>>>>>>>>>>
> > >>>>>>>>>>> I'd like to get a discussion going around some of the API
> > choices
> > >>>>>> we've
> > >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
> > >>>>>> operations
> > >>>>>>>>>>> (though this could expand).
> > >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the
> API,
> > >>>> i.e,
> > >>>>>>>> there
> > >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> > >>>> noisy
> > >>>>>> and
> > >>>>>>>> i
> > >>>>>>>>>>> feel it is only going to get worse as we add more optional
> > >>>> params.
> > >>>>> In
> > >>>>>>>>>>> particular we've had some requests to be able to turn caching
> > >>>> off,
> > >>>>> or
> > >>>>>>>>>>> change log configs,  on a per operator basis (note this can
> be
> > >>>> done
> > >>>>>> now
> > >>>>>>>>>> if
> > >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> > >>>>> cumbersome).
> > >>>>>>>>>>> So this is a bit of an open question. How can we change the
> DSL
> > >>>>>>>> overloads
> > >>>>>>>>>>> so that it flows, is simple to use and understand, and is
> > easily
> > >>>>>>>> extended
> > >>>>>>>>>>> in the future?
> > >>>>>>>>>>>
> > >>>>>>>>>>> One option would be to use a fluent API approach for
> providing
> > >>>> the
> > >>>>>>>>>> optional
> > >>>>>>>>>>> params, so something like this:
> > >>>>>>>>>>>
> > >>>>>>>>>>> groupedStream.count()
> > >>>>>>>>>>>     .withStoreName("name")
> > >>>>>>>>>>>     .withCachingEnabled(false)
> > >>>>>>>>>>>     .withLoggingEnabled(config)
> > >>>>>>>>>>>     .table()
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>> Another option would be to provide a Builder to the count
> > method,
> > >>>>> so
> > >>>>>> it
> > >>>>>>>>>>> would look something like this:
> > >>>>>>>>>>> groupedStream.count(new
> > >>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>>>>>>>>>>
> > >>>>>>>>>>> Another option is to say: Hey we don't need this, what are
> you
> > on
> > >>>>>>>> about!
> > >>>>>>>>>>> The above has focussed on state store related overloads, but
> > the
> > >>>>> same
> > >>>>>>>>>> ideas
> > >>>>>>>>>>> could  be applied to joins etc, where we presently have many
> > join
> > >>>>>>>> methods
> > >>>>>>>>>>> and many overloads.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks,
> > >>>>>>>>>>> Damian
> > >>>>>>>>>>>
> > >>>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>> --
> > >>>>> -- Guozhang
> > >>>>>
> > >>>>>
> > >>>>
> > >>
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Jan,

Thanks very much for the input.

On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com> wrote:

> Hi Damian,
>
> I do see your point of something needs to change. But I fully agree with
> Gouzhang when he says.
> ---
>
> But since this is a incompatibility change, and we are going to remove the
> compatibility annotations soon it means we only have one chance and we
> really have to make it right.
> ----
>
>
I think we all agree on this one! Hence the discussion.


> I fear all suggestions do not go far enough to become something that will
> carry on for very much longer.
> I am currently working on KAFKA-3705 and try to find the most easy way for
> the user to give me all the required functionality. The easiest interface I
> could come up so far can be looked at here.
>
>
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L622
>
>
And its already horribly complicated. I am currently unable to find the
> right abstraction level to have everything falling into place naturally. To
> be honest I already think introducing
>
>
To be fair that is not a particularly easy problem to solve!


>
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L493
>
> was unideal and makes everything a mess.


I'm not sure i agree that it makes everything a mess, but It could have
been done differently.

The JoinType:Whatever is also not really flexible. 2 things come to my mind:
>
> 1. I don't think we should rule out config based decisions say configs like
>         streams.$applicationID.joins.$joinname.conf = value
>

Is this just for config? Or are you suggesting that we could somehow "code"
the join in a config file?


> This can allow for tremendous changes without single API change and IMO it
> was not considered enough yet.
>
> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
> example can be used to implement different join types as the user wishes.
>

Do you have an example of how this might look?


> As Gouzhang said: stopping to break users is very important.


Of course. We want to make it as easy as possible for people to use
streams.


especially with this changes + All the plans I sadly only have in my head
> but hopefully the first link can give a glimpse.
>
> Thanks for preparing the examples made it way clearer to me what exactly
> we are talking about. I would argue to go a bit slower and more carefull on
> this one. At some point we need to get it right. Peeking over to the hadoop
> guys with their hughe userbase. Config files really work well for them.
>
> Best Jan
>
>
>
>
>
> On 30.06.2017 09:31, Damian Guy wrote:
> > Thanks Matthias
> >
> > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
> wrote:
> >
> >> I am just catching up on this thread, so sorry for the long email in
> >> advance... Also, it's to some extend a dump of thoughts and not always a
> >> clear proposal. Still need to think about this in more detail. But maybe
> >> it helps other to get new ideas :)
> >>
> >>
> >>>> However, I don't understand your argument about putting aggregate()
> >>>> after the withXX() -- all the calls to withXX() set optional
> parameters
> >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >>>> be quite confusion for developers.
> >>>>
> >>>>
> >>> I see what you are saying, but the grouped stream is effectively a
> no-op
> >>> until you call one of the aggregate/count/reduce etc functions. So the
> >>> optional params are ones that are applicable to any of the operations
> you
> >>> can perform on this grouped stream. Then the final
> >>> count()/reduce()/aggregate() call has any of the params that are
> >>> required/specific to that function.
> >>>
> >> I understand your argument, but you don't share the conclusion. If we
> >> need a "final/terminal" call, the better way might be
> >>
> >> .groupBy().count().withXX().build()
> >>
> >> (with a better name for build() though)
> >>
> >>
> > The point is that all the other calls, i.e,withBlah, windowed, etc apply
> > too all the aggregate functions. The terminal call being the actual type
> of
> > aggregation you want to do. I personally find this more natural than
> > groupBy().count().withBlah().build()
> >
> >
> >>> groupedStream.count(/** non windowed count**/)
> >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>
> >> I like this. However, I don't see a reason to have windowed() and
> >> sessionWindowed(). We should have one top-level `Windows` interface that
> >> both `TimeWindows` and `SessionWindows` implement and just have a single
> >> windowed() method that accepts all `Windows`. (I did not like the
> >> separation of `SessionWindows` in the first place, and this seems to be
> >> an opportunity to clean this up. It was hard to change when we
> >> introduced session windows)
> >>
> > Yes - true we should look into that.
> >
> >
> >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> >> might also want to use windowBy() (instead of windowed()). Not sure how
> >> important this is, but it seems to be inconsistent otherwise.
> >>
> >>
> > Makes sense
> >
> >
> >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
> >> defining an inner/left/outer join is not an optional argument but a
> >> first class concept and should have a proper representation in the API
> >> (like the current methods join(), leftJoin, outerJoin()).
> >>
> >>
> > Yep, i did originally have it as a required param and maybe that is what
> we
> > go with. It could have a default, but maybe that is confusing.
> >
> >
> >
> >> About the two join API proposals, the second one has too much boiler
> >> plate code for my taste. Also, the actual join() operator has only one
> >> argument what is weird to me, as in my thinking process, the main
> >> operator call, should have one parameter per mandatory argument but your
> >> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
> >> This is far from intuitive IMHO.
> >>
> >>
> > This is the builder pattern, you only need one param as the builder has
> > captured all of the required and optional arguments.
> >
> >
> >> The first join proposal also seems to align better with the pattern
> >> suggested for aggregations and having the same pattern for all operators
> >> is important (as you stated already).
> >>
> >>
> > This is why i offered two alternatives as i started out with. 1 is the
> > builder pattern, the other is the more fluent pattern.
> >
> >
> >>
> >> Coming back to the config vs optional parameter. What about having a
> >> method withConfig[s](...) that allow to put in the configuration?
> >>
> >>
> > Sure, it is currently called withLogConfig() as that is the only thing
> that
> > is really config.
> >
> >
> >> This also raises the question if until() is a windows property?
> >> Actually, until() seems to be a configuration parameter and thus, should
> >> not not have it's own method.
> >>
> >>
> > Hmmm, i don't agree. Until is a property of the window. It is going to be
> > potentially different for every window operation you do in a streams app.
> >
> >
> >>
> >> Browsing throw your example DSL branch, I also saw this one:
> >>
> >>> final KTable<Windowed<String>, Long> windowed>
> >>   groupedStream.counting()
> >>>                   .windowed(TimeWindows.of(10L).until(10))
> >>>                   .table();
> >> This is an interesting idea, and it remind my on some feedback about "I
> >> wanted to count a stream, but there was no count() method -- I first
> >> needed to figure out, that I need to group the stream first to be able
> >> to count it. It does make sense in hindsight but was not obvious in the
> >> beginning". Thus, carrying out this thought, we could also do the
> >> following:
> >>
> >> stream.count().groupedBy().windowedBy().table();
> >>
> >> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> >> it comes after the count()
> >>
> >> This would be more consistent than your proposal (that has grouping
> >> before but windowing after count()). It might even allow us to enrich
> >> the API with a some syntactic sugar like `stream.count().table()` to get
> >> the overall count of all records (this would obviously not scale, but we
> >> could support it -- if not now, maybe later).
> >>
> >>
> > I guess i'd prefer
> > stream.groupBy().windowBy().count()
> > stream.groupBy().windowBy().reduce()
> > stream.groupBy().count()
> >
> > As i said above, everything that happens before the final aggregate call
> > can be applied to any of them. So it makes sense to me to do those things
> > ahead of the final aggregate call.
> >
> >
> >> Last about builder pattern. I am convinced that we need some "terminal"
> >> operator/method that tells us when to add the processor to the topology.
> >> But I don't see the need for a plain builder pattern that feels alien to
> >> me (see my argument about the second join proposal). Using .stream() /
> >> .table() as use in many examples might work. But maybe a more generic
> >> name that we can use in all places like build() or apply() might also be
> >> an option.
> >>
> >>
> > Sure, a generic name might be ok.
> >
> >
> >
> >
> >> -Matthias
> >>
> >>
> >>
> >> On 6/29/17 7:37 AM, Damian Guy wrote:
> >>> Thanks Kyle.
> >>>
> >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Damian,
> >>>>
> >>>>>>>> When trying to program in the fluent API that has been discussed
> >> most
> >>>> it
> >>>>>>>> feels difficult to know when you will actually get an object you
> can
> >>>> reuse.
> >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
> legal
> >> to
> >>>>>>>> reuse it or does this approach expect you to call grouped each
> time?
> >>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> >> as
> >>>> you
> >>>>>> can today.
> >>>> You said it yourself in another post that the grouped stream is
> >>>> effectively a no-op until a count, reduce, or aggregate. The way I see
> >> it
> >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
> >> because
> >>>> most of this fluent api would continue returning this (this being the
> >>>> builder object currently being manipulated).
> >>> So, if you ever store a reference to anything but KStreams and KTables
> >> and
> >>>> you use it in two different ways then its possible you make
> conflicting
> >>>> withXXX() calls on the same builder.
> >>>>
> >>>>
> >>> No necessarily true. It could return a new instance of the builder,
> i.e.,
> >>> the builders being immutable. So if you held a reference to the builder
> >> it
> >>> would always be the same as it was when it was created.
> >>>
> >>>
> >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >>>>
> >>>> I’ll admit that this shouldn’t happen but some user is going to do it
> >>>> eventually…
> >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
> would
> >>>> most likely be equivalent to the version withDeclaredSerdes. One work
> >>>> around would be to always make copies of the config objects you are
> >>>> building, but this approach has its own problem because now we have to
> >>>> identify which configs are equivalent so we don’t create repeated
> >>>> processors.
> >>>>
> >>>> The point of this long winded example is that we always have to be
> >>>> thinking about all of the possible ways it could be misused by a user
> >>>> (causing them to see hard to diagnose problems).
> >>>>
> >>> Exactly! That is the point of the discussion really.
> >>>
> >>>
> >>>> In my attempt at a couple methods with builders I feel that I could
> >>>> confidently say the user couldn’t really mess it up.
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>> The kGroupedStream is reusable and if they attempted to reuse the
> Count
> >>>> for some reason it would throw an error message saying that a store
> >> named
> >>>> “my-store” already exists.
> >>>>
> >>>>
> >>> Yes i agree and i think using builders is my preferred pattern.
> >>>
> >>> Cheers,
> >>> Damian
> >>>
> >>>
> >>>> Thanks,
> >>>> Kyle
> >>>>
> >>>> From: Damian Guy
> >>>> Sent: Thursday, June 29, 2017 3:59 AM
> >>>> To: dev@kafka.apache.org
> >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>
> >>>> Hi Kyle,
> >>>>
> >>>> Thanks for your input. Really appreciated.
> >>>>
> >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <winkelman.kyle@gmail.com
> >
> >>>> wrote:
> >>>>
> >>>>> I like more of a builder pattern even though others have voiced
> against
> >>>>> it. The reason I like it is because it makes it clear to the user
> that
> >> a
> >>>>> call to KGroupedStream#count will return a KTable not some
> intermediate
> >>>>> class that I need to undetstand.
> >>>>>
> >>>> Yes, that makes sense.
> >>>>
> >>>>
> >>>>> When trying to program in the fluent API that has been discussed most
> >> it
> >>>>> feels difficult to know when you will actually get an object you can
> >>>> reuse.
> >>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> to
> >>>>> reuse it or does this approach expect you to call grouped each time?
> >>>>
> >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> as
> >> you
> >>>> can today.
> >>>>
> >>>>
> >>>>> This question doesn’t pop into my head at all in the builder pattern
> I
> >>>>> assume I can reuse everything.
> >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
> fan
> >> of
> >>>>> the grouped.
> >>>>>
> >>>>> Yes, grouped() was more for demonstration and because groupBy() and
> >>>> groupByKey() were taken! So i'd imagine the api would actually want to
> >> be
> >>>> groupByKey(/** no required args***/).withOptionalArg() and
> >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> >> depends
> >>>> on maintaining backward compatibility.
> >>>>
> >>>>
> >>>>> Unfortunately, the below approach would require atleast 2 (probably
> 3)
> >>>>> overloads (one for returning a KTable and one for returning a KTable
> >> with
> >>>>> Windowed Key, probably would want to split windowed and
> sessionwindowed
> >>>> for
> >>>>> ease of implementation) of each count, reduce, and aggregate.
> >>>>> Obviously not exhaustive but enough for you to get the picture.
> Count,
> >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
> >> builder:
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Windowed Count
> >>>>> KTable<Windowed<String>, Long> windowedCount =
> >>>>>
> >>
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >>>>> // Session Count
> >>>>> KTable<Windowed<String>, Long> sessionCount =
> >>>>>
> >>
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >>>>>
> >>>> Above and below, i think i'd prefer it to be:
> >>>> groupedStream.count(/** non windowed count**/)
> >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>> // Reduce
> >>>>> Reducer<Long> reducer;
> >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Aggregate Windowed with Custom Store
> >>>>> Initializer<String> initializer;
> >>>>> Aggregator<String, Long, String> aggregator;
> >>>>> KTable<Windowed<String>, String> aggregate =
> >>>>> groupedStream.aggregate(initializer, aggregator,
> >>>>>
> >>
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >>>>> // Cogroup SessionWindowed
> >>>>> KTable<String, String> cogrouped =
> groupedStream1.cogroup(aggregator1)
> >>>>>          .cogroup(groupedStream2, aggregator2)
> >>>>>          .aggregate(initializer, aggregator,
> >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> >>>>> sessionMerger).withQueryableStoreName("my-store"));
> >>>>>
> >>>>>
> >>>>>
> >>>>> public class Count {
> >>>>>
> >>>>>      public static class Windowed extends Count {
> >>>>>          private Windows windows;
> >>>>>      }
> >>>>>      public static class SessionWindowed extends Count {
> >>>>>          private SessionWindows sessionWindows;
> >>>>>      }
> >>>>>
> >>>>>      public static Count count();
> >>>>>      public static Windowed windowed(Windows windows);
> >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
> >>>>> sessionWindows);
> >>>>>
> >>>>>      // All withXXX(...) methods.
> >>>>> }
> >>>>>
> >>>>> public class KGroupedStream {
> >>>>>      public KTable<K, Long> count(Count count);
> >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> >> count);
> >>>>> …
> >>>>> }
> >>>>>
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> From: Guozhang Wang
> >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
> >>>>> To: dev@kafka.apache.org
> >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>>
> >>>>> I played the current proposal a bit with
> >> https://github.com/dguy/kafka/
> >>>>> tree/dsl-experiment <
> https://github.com/dguy/kafka/tree/dsl-experiment
> >>> ,
> >>>>> and here are my observations:
> >>>>>
> >>>>> 1. Personally I prefer
> >>>>>
> >>>>>      "stream.group(mapper) / stream.groupByKey()"
> >>>>>
> >>>>> than
> >>>>>
> >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
> >>>>>
> >>>>> Since 1) withKeyMapper is not enforced programmatically though it is
> >> not
> >>>>> "really" optional like others, 2) syntax-wise it reads more natural.
> >>>>>
> >>>>> I think it is okay to add the APIs in (
> >>>>>
> >>>>>
> >>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> >>>>> )
> >>>>> in KGroupedStream.
> >>>>>
> >>>>>
> >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
> pass
> >> in
> >>>>> the most-inner state store supplier (e.g. then one whose get() return
> >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
> with
> >>>>> logging / metrics / etc? I think it would be more useful to only
> >> require
> >>>>> users pass in the inner state store supplier while specifying
> caching /
> >>>>> logging through other APIs.
> >>>>>
> >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me:
> we
> >>>> are
> >>>>> allowing users to call other APIs like "withQueryableName" multiple
> >> time,
> >>>>> but only call "withStateStoreSupplier" only once in the end. Why is
> >> that?
> >>>>>
> >>>>> 3. The current DSL seems to be only for aggregations, what about
> joins?
> >>>>>
> >>>>>
> >>>>> 4. I think it is okay to keep the "withLogConfig": for the
> >>>>> StateStoreSupplier it will still be user code specifying the topology
> >> so
> >>>> I
> >>>>> do not see there is a big difference.
> >>>>>
> >>>>>
> >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> >>>>> windowed state store supplier to enforce typing?
> >>>>>
> >>>>>
> >>>>> Below are minor ones:
> >>>>>
> >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> >>>>>
> >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >>>>>
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> >> matthias@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> I see your point about "when to add the processor to the topology".
> >>>> That
> >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
> >>>>> topology...
> >>>>>> I don't see any problem with having all the withXX() in KTable
> >>>> interface
> >>>>>> -- but this might be subjective.
> >>>>>>
> >>>>>>
> >>>>>> However, I don't understand your argument about putting aggregate()
> >>>>>> after the withXX() -- all the calls to withXX() set optional
> >> parameters
> >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> might
> >>>>>> be quite confusion for developers.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> >>>>>>>> I also think that mixing optional parameters with configs is a bad
> >>>>> idea.
> >>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
> >>>> to
> >>>>>>>> find some time to come up with something.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> Yes, i don't like the mix of config either. But the only real
> config
> >>>>> here
> >>>>>>> is the logging config - which we don't really need as it can
> already
> >>>> be
> >>>>>>> done via a custom StateStoreSupplier.
> >>>>>>>
> >>>>>>>
> >>>>>>>> What I don't like in the current proposal is the
> >>>>>>>> .grouped().withKeyMapper() -- the current solution with
> >>>> .groupBy(...)
> >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
> >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
> find
> >>>>>>>> some better names).
> >>>>>>>>
> >>>>>>>>
> >>>>>>> it could be groupByKey(), groupBy() or something different bt
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>> The proposed pattern "chains" grouping and aggregation too close
> >>>>>>>> together. I would rather separate both more than less, ie, do into
> >>>> the
> >>>>>>>> opposite direction.
> >>>>>>>>
> >>>>>>>> I am also wondering, if we could so something more "fluent". The
> >>>>> initial
> >>>>>>>> proposal was like:
> >>>>>>>>
> >>>>>>>>>> groupedStream.count()
> >>>>>>>>>>     .withStoreName("name")
> >>>>>>>>>>     .withCachingEnabled(false)
> >>>>>>>>>>     .withLoggingEnabled(config)
> >>>>>>>>>>     .table()
> >>>>>>>> The .table() statement in the end was kinda alien.
> >>>>>>>>
> >>>>>>> I agree, but then all of the withXXX methods need to be on KTable
> >>>> which
> >>>>>> is
> >>>>>>> worse in my opinion. You also need something that is going to
> "build"
> >>>>> the
> >>>>>>> internal processors and add them to the topology.
> >>>>>>>
> >>>>>>>
> >>>>>>>> The current proposal put the count() into the end -- ie, the
> >>>> optional
> >>>>>>>> parameter for count() have to specified on the .grouped() call --
> >>>> this
> >>>>>>>> does not seems to be the best way either.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> I actually prefer this method as you are building a grouped stream
> >>>> that
> >>>>>> you
> >>>>>>> will aggregate. So
> >>>> table.grouped(...).withOptionalStuff().aggregate(..)
> >>>>>> etc
> >>>>>>> seems natural to me.
> >>>>>>>
> >>>>>>>
> >>>>>>>> I did not think this through in detail, but can't we just do the
> >>>>> initial
> >>>>>>>> proposal with the .table() ?
> >>>>>>>>
> >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> >>>>>>>>
> >>>>>>>> Each .withXXX(...) return the current KTable and all the
> .withXXX()
> >>>>> are
> >>>>>>>> just added to the KTable interface. Or do I miss anything why this
> >>>>> wont'
> >>>>>>>> work or any obvious disadvantage?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>> See above.
> >>>>>>>
> >>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> >>>> fluent
> >>>>>>>>> approach, but i think it is slightly nicer.
> >>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
> >>>> the
> >>>>>> DSL,
> >>>>>>>>> but i think that enabling caching and enabling logging are things
> >>>>> that
> >>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> >>>> (even
> >>>>>>>>> though it is below) as this is actually config and we already
> have
> >>>> a
> >>>>>> way
> >>>>>>>> of
> >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
> that
> >>>>> is
> >>>>>> a
> >>>>>>>>> bit of a tedious process for someone that just wants to use the
> >>>>> default
> >>>>>>>>> storage engine, but not have caching enabled.
> >>>>>>>>>
> >>>>>>>>> There is also an orthogonal concern that Guozhang alluded to....
> If
> >>>>> you
> >>>>>>>>> want to plug in a custom storage engine and you want it to be
> >>>> logged
> >>>>>> etc,
> >>>>>>>>> you would currently need to implement that yourself. Ideally we
> can
> >>>>>>>> provide
> >>>>>>>>> a way where we will wrap the custom store with logging, metrics,
> >>>>> etc. I
> >>>>>>>>> need to think about where this fits, it is probably more
> >>>> appropriate
> >>>>> on
> >>>>>>>> the
> >>>>>>>>> Stores API.
> >>>>>>>>>
> >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>>>>>>>> // count with mapped key
> >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
> >>>>>>>>>          .withKeyMapper(keyMapper)
> >>>>>>>>>          .withKeySerde(Serdes.Long())
> >>>>>>>>>          .withValueSerde(Serdes.String())
> >>>>>>>>>          .withQueryableName("my-store")
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // windowed count
> >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> >>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("my-window-store")
> >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // windowed reduce
> >>>>>>>>> final Reducer<String> windowedReducer = null;
> >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> >>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("my-window-store")
> >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>          .reduce(windowedReducer);
> >>>>>>>>>
> >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
> >>>>>>>>> final Initializer<Long> init = null;
> >>>>>>>>>
> >>>>>>>>> // aggregate
> >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> >>>>>>>>>          .withQueryableName("my-aggregate-store")
> >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
> >>>>>>>>>
> >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> >>>>>> stateStoreSupplier
> >>>>>>>> = null;
> >>>>>>>>> // aggregate with custom store
> >>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
> >>>>>>>>>          .aggregate(aggregator, init);
> >>>>>>>>>
> >>>>>>>>> // disable caching
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("name")
> >>>>>>>>>          .withCachingEnabled(false)
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // disable logging
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("q")
> >>>>>>>>>          .withLoggingEnabled(false)
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // override log config
> >>>>>>>>> final Reducer<String> reducer = null;
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withLogConfig(Collections.singletonMap("segment.size",
> >>>>> "10"))
> >>>>>>>>>          .reduce(reducer);
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> If anyone wants to play around with this you can find the code
> >>>> here:
> >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>>>>>>>
> >>>>>>>>> Note: It won't actually work as most of the methods just return
> >>>> null.
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> >>>> wrote:
> >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
> >>>> are
> >>>>>>>> better
> >>>>>>>>>> than overload abuse.
> >>>>>>>>>>
> >>>>>>>>>> The fluent API approach reads better, no mention of builder or
> >>>> build
> >>>>>>>>>> anywhere. The main downside is that the method signatures are a
> >>>>> little
> >>>>>>>> less
> >>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
> >>>>> knows
> >>>>>>>> what
> >>>>>>>>>> it returns. Also, one needs to figure out the special method
> >>>>>> (`table()`
> >>>>>>>> in
> >>>>>>>>>> this case) that gives you what you actually care about (`KTable`
> >>>> in
> >>>>>> this
> >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
> >>>>>>>> comparison.
> >>>>>>>>>> The builder approach avoids the issues mentioned above, but it
> >>>>> doesn't
> >>>>>>>> read
> >>>>>>>>>> as well.
> >>>>>>>>>>
> >>>>>>>>>> Ismael
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
> damian.guy@gmail.com
> >>>>>>>> wrote:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> I'd like to get a discussion going around some of the API
> choices
> >>>>>> we've
> >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
> >>>>>> operations
> >>>>>>>>>>> (though this could expand).
> >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
> >>>> i.e,
> >>>>>>>> there
> >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> >>>> noisy
> >>>>>> and
> >>>>>>>> i
> >>>>>>>>>>> feel it is only going to get worse as we add more optional
> >>>> params.
> >>>>> In
> >>>>>>>>>>> particular we've had some requests to be able to turn caching
> >>>> off,
> >>>>> or
> >>>>>>>>>>> change log configs,  on a per operator basis (note this can be
> >>>> done
> >>>>>> now
> >>>>>>>>>> if
> >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> >>>>> cumbersome).
> >>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
> >>>>>>>> overloads
> >>>>>>>>>>> so that it flows, is simple to use and understand, and is
> easily
> >>>>>>>> extended
> >>>>>>>>>>> in the future?
> >>>>>>>>>>>
> >>>>>>>>>>> One option would be to use a fluent API approach for providing
> >>>> the
> >>>>>>>>>> optional
> >>>>>>>>>>> params, so something like this:
> >>>>>>>>>>>
> >>>>>>>>>>> groupedStream.count()
> >>>>>>>>>>>     .withStoreName("name")
> >>>>>>>>>>>     .withCachingEnabled(false)
> >>>>>>>>>>>     .withLoggingEnabled(config)
> >>>>>>>>>>>     .table()
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Another option would be to provide a Builder to the count
> method,
> >>>>> so
> >>>>>> it
> >>>>>>>>>>> would look something like this:
> >>>>>>>>>>> groupedStream.count(new
> >>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>>>>>>>
> >>>>>>>>>>> Another option is to say: Hey we don't need this, what are you
> on
> >>>>>>>> about!
> >>>>>>>>>>> The above has focussed on state store related overloads, but
> the
> >>>>> same
> >>>>>>>>>> ideas
> >>>>>>>>>>> could  be applied to joins etc, where we presently have many
> join
> >>>>>>>> methods
> >>>>>>>>>>> and many overloads.
> >>>>>>>>>>>
> >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Damian
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>>
> >>>>
> >>
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Jan,

Thanks very much for the input.

On Tue, 4 Jul 2017 at 08:54 Jan Filipiak <Ja...@trivago.com> wrote:

> Hi Damian,
>
> I do see your point of something needs to change. But I fully agree with
> Gouzhang when he says.
> ---
>
> But since this is a incompatibility change, and we are going to remove the
> compatibility annotations soon it means we only have one chance and we
> really have to make it right.
> ----
>
>
I think we all agree on this one! Hence the discussion.


> I fear all suggestions do not go far enough to become something that will
> carry on for very much longer.
> I am currently working on KAFKA-3705 and try to find the most easy way for
> the user to give me all the required functionality. The easiest interface I
> could come up so far can be looked at here.
>
>
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L622
>
>
And its already horribly complicated. I am currently unable to find the
> right abstraction level to have everything falling into place naturally. To
> be honest I already think introducing
>
>
To be fair that is not a particularly easy problem to solve!


>
> https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L493
>
> was unideal and makes everything a mess.


I'm not sure i agree that it makes everything a mess, but It could have
been done differently.

The JoinType:Whatever is also not really flexible. 2 things come to my mind:
>
> 1. I don't think we should rule out config based decisions say configs like
>         streams.$applicationID.joins.$joinname.conf = value
>

Is this just for config? Or are you suggesting that we could somehow "code"
the join in a config file?


> This can allow for tremendous changes without single API change and IMO it
> was not considered enough yet.
>
> 2. Push logic from the DSL to the Callback classes. A ValueJoiner for
> example can be used to implement different join types as the user wishes.
>

Do you have an example of how this might look?


> As Gouzhang said: stopping to break users is very important.


Of course. We want to make it as easy as possible for people to use
streams.


especially with this changes + All the plans I sadly only have in my head
> but hopefully the first link can give a glimpse.
>
> Thanks for preparing the examples made it way clearer to me what exactly
> we are talking about. I would argue to go a bit slower and more carefull on
> this one. At some point we need to get it right. Peeking over to the hadoop
> guys with their hughe userbase. Config files really work well for them.
>
> Best Jan
>
>
>
>
>
> On 30.06.2017 09:31, Damian Guy wrote:
> > Thanks Matthias
> >
> > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
> wrote:
> >
> >> I am just catching up on this thread, so sorry for the long email in
> >> advance... Also, it's to some extend a dump of thoughts and not always a
> >> clear proposal. Still need to think about this in more detail. But maybe
> >> it helps other to get new ideas :)
> >>
> >>
> >>>> However, I don't understand your argument about putting aggregate()
> >>>> after the withXX() -- all the calls to withXX() set optional
> parameters
> >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >>>> be quite confusion for developers.
> >>>>
> >>>>
> >>> I see what you are saying, but the grouped stream is effectively a
> no-op
> >>> until you call one of the aggregate/count/reduce etc functions. So the
> >>> optional params are ones that are applicable to any of the operations
> you
> >>> can perform on this grouped stream. Then the final
> >>> count()/reduce()/aggregate() call has any of the params that are
> >>> required/specific to that function.
> >>>
> >> I understand your argument, but you don't share the conclusion. If we
> >> need a "final/terminal" call, the better way might be
> >>
> >> .groupBy().count().withXX().build()
> >>
> >> (with a better name for build() though)
> >>
> >>
> > The point is that all the other calls, i.e,withBlah, windowed, etc apply
> > too all the aggregate functions. The terminal call being the actual type
> of
> > aggregation you want to do. I personally find this more natural than
> > groupBy().count().withBlah().build()
> >
> >
> >>> groupedStream.count(/** non windowed count**/)
> >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>
> >> I like this. However, I don't see a reason to have windowed() and
> >> sessionWindowed(). We should have one top-level `Windows` interface that
> >> both `TimeWindows` and `SessionWindows` implement and just have a single
> >> windowed() method that accepts all `Windows`. (I did not like the
> >> separation of `SessionWindows` in the first place, and this seems to be
> >> an opportunity to clean this up. It was hard to change when we
> >> introduced session windows)
> >>
> > Yes - true we should look into that.
> >
> >
> >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> >> might also want to use windowBy() (instead of windowed()). Not sure how
> >> important this is, but it seems to be inconsistent otherwise.
> >>
> >>
> > Makes sense
> >
> >
> >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
> >> defining an inner/left/outer join is not an optional argument but a
> >> first class concept and should have a proper representation in the API
> >> (like the current methods join(), leftJoin, outerJoin()).
> >>
> >>
> > Yep, i did originally have it as a required param and maybe that is what
> we
> > go with. It could have a default, but maybe that is confusing.
> >
> >
> >
> >> About the two join API proposals, the second one has too much boiler
> >> plate code for my taste. Also, the actual join() operator has only one
> >> argument what is weird to me, as in my thinking process, the main
> >> operator call, should have one parameter per mandatory argument but your
> >> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
> >> This is far from intuitive IMHO.
> >>
> >>
> > This is the builder pattern, you only need one param as the builder has
> > captured all of the required and optional arguments.
> >
> >
> >> The first join proposal also seems to align better with the pattern
> >> suggested for aggregations and having the same pattern for all operators
> >> is important (as you stated already).
> >>
> >>
> > This is why i offered two alternatives as i started out with. 1 is the
> > builder pattern, the other is the more fluent pattern.
> >
> >
> >>
> >> Coming back to the config vs optional parameter. What about having a
> >> method withConfig[s](...) that allow to put in the configuration?
> >>
> >>
> > Sure, it is currently called withLogConfig() as that is the only thing
> that
> > is really config.
> >
> >
> >> This also raises the question if until() is a windows property?
> >> Actually, until() seems to be a configuration parameter and thus, should
> >> not not have it's own method.
> >>
> >>
> > Hmmm, i don't agree. Until is a property of the window. It is going to be
> > potentially different for every window operation you do in a streams app.
> >
> >
> >>
> >> Browsing throw your example DSL branch, I also saw this one:
> >>
> >>> final KTable<Windowed<String>, Long> windowed>
> >>   groupedStream.counting()
> >>>                   .windowed(TimeWindows.of(10L).until(10))
> >>>                   .table();
> >> This is an interesting idea, and it remind my on some feedback about "I
> >> wanted to count a stream, but there was no count() method -- I first
> >> needed to figure out, that I need to group the stream first to be able
> >> to count it. It does make sense in hindsight but was not obvious in the
> >> beginning". Thus, carrying out this thought, we could also do the
> >> following:
> >>
> >> stream.count().groupedBy().windowedBy().table();
> >>
> >> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> >> it comes after the count()
> >>
> >> This would be more consistent than your proposal (that has grouping
> >> before but windowing after count()). It might even allow us to enrich
> >> the API with a some syntactic sugar like `stream.count().table()` to get
> >> the overall count of all records (this would obviously not scale, but we
> >> could support it -- if not now, maybe later).
> >>
> >>
> > I guess i'd prefer
> > stream.groupBy().windowBy().count()
> > stream.groupBy().windowBy().reduce()
> > stream.groupBy().count()
> >
> > As i said above, everything that happens before the final aggregate call
> > can be applied to any of them. So it makes sense to me to do those things
> > ahead of the final aggregate call.
> >
> >
> >> Last about builder pattern. I am convinced that we need some "terminal"
> >> operator/method that tells us when to add the processor to the topology.
> >> But I don't see the need for a plain builder pattern that feels alien to
> >> me (see my argument about the second join proposal). Using .stream() /
> >> .table() as use in many examples might work. But maybe a more generic
> >> name that we can use in all places like build() or apply() might also be
> >> an option.
> >>
> >>
> > Sure, a generic name might be ok.
> >
> >
> >
> >
> >> -Matthias
> >>
> >>
> >>
> >> On 6/29/17 7:37 AM, Damian Guy wrote:
> >>> Thanks Kyle.
> >>>
> >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Damian,
> >>>>
> >>>>>>>> When trying to program in the fluent API that has been discussed
> >> most
> >>>> it
> >>>>>>>> feels difficult to know when you will actually get an object you
> can
> >>>> reuse.
> >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
> legal
> >> to
> >>>>>>>> reuse it or does this approach expect you to call grouped each
> time?
> >>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> >> as
> >>>> you
> >>>>>> can today.
> >>>> You said it yourself in another post that the grouped stream is
> >>>> effectively a no-op until a count, reduce, or aggregate. The way I see
> >> it
> >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
> >> because
> >>>> most of this fluent api would continue returning this (this being the
> >>>> builder object currently being manipulated).
> >>> So, if you ever store a reference to anything but KStreams and KTables
> >> and
> >>>> you use it in two different ways then its possible you make
> conflicting
> >>>> withXXX() calls on the same builder.
> >>>>
> >>>>
> >>> No necessarily true. It could return a new instance of the builder,
> i.e.,
> >>> the builders being immutable. So if you held a reference to the builder
> >> it
> >>> would always be the same as it was when it was created.
> >>>
> >>>
> >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >>>>
> >>>> I’ll admit that this shouldn’t happen but some user is going to do it
> >>>> eventually…
> >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
> would
> >>>> most likely be equivalent to the version withDeclaredSerdes. One work
> >>>> around would be to always make copies of the config objects you are
> >>>> building, but this approach has its own problem because now we have to
> >>>> identify which configs are equivalent so we don’t create repeated
> >>>> processors.
> >>>>
> >>>> The point of this long winded example is that we always have to be
> >>>> thinking about all of the possible ways it could be misused by a user
> >>>> (causing them to see hard to diagnose problems).
> >>>>
> >>> Exactly! That is the point of the discussion really.
> >>>
> >>>
> >>>> In my attempt at a couple methods with builders I feel that I could
> >>>> confidently say the user couldn’t really mess it up.
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>> The kGroupedStream is reusable and if they attempted to reuse the
> Count
> >>>> for some reason it would throw an error message saying that a store
> >> named
> >>>> “my-store” already exists.
> >>>>
> >>>>
> >>> Yes i agree and i think using builders is my preferred pattern.
> >>>
> >>> Cheers,
> >>> Damian
> >>>
> >>>
> >>>> Thanks,
> >>>> Kyle
> >>>>
> >>>> From: Damian Guy
> >>>> Sent: Thursday, June 29, 2017 3:59 AM
> >>>> To: dev@kafka.apache.org
> >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>
> >>>> Hi Kyle,
> >>>>
> >>>> Thanks for your input. Really appreciated.
> >>>>
> >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <winkelman.kyle@gmail.com
> >
> >>>> wrote:
> >>>>
> >>>>> I like more of a builder pattern even though others have voiced
> against
> >>>>> it. The reason I like it is because it makes it clear to the user
> that
> >> a
> >>>>> call to KGroupedStream#count will return a KTable not some
> intermediate
> >>>>> class that I need to undetstand.
> >>>>>
> >>>> Yes, that makes sense.
> >>>>
> >>>>
> >>>>> When trying to program in the fluent API that has been discussed most
> >> it
> >>>>> feels difficult to know when you will actually get an object you can
> >>>> reuse.
> >>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> to
> >>>>> reuse it or does this approach expect you to call grouped each time?
> >>>>
> >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> as
> >> you
> >>>> can today.
> >>>>
> >>>>
> >>>>> This question doesn’t pop into my head at all in the builder pattern
> I
> >>>>> assume I can reuse everything.
> >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
> fan
> >> of
> >>>>> the grouped.
> >>>>>
> >>>>> Yes, grouped() was more for demonstration and because groupBy() and
> >>>> groupByKey() were taken! So i'd imagine the api would actually want to
> >> be
> >>>> groupByKey(/** no required args***/).withOptionalArg() and
> >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> >> depends
> >>>> on maintaining backward compatibility.
> >>>>
> >>>>
> >>>>> Unfortunately, the below approach would require atleast 2 (probably
> 3)
> >>>>> overloads (one for returning a KTable and one for returning a KTable
> >> with
> >>>>> Windowed Key, probably would want to split windowed and
> sessionwindowed
> >>>> for
> >>>>> ease of implementation) of each count, reduce, and aggregate.
> >>>>> Obviously not exhaustive but enough for you to get the picture.
> Count,
> >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
> >> builder:
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Windowed Count
> >>>>> KTable<Windowed<String>, Long> windowedCount =
> >>>>>
> >>
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >>>>> // Session Count
> >>>>> KTable<Windowed<String>, Long> sessionCount =
> >>>>>
> >>
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >>>>>
> >>>> Above and below, i think i'd prefer it to be:
> >>>> groupedStream.count(/** non windowed count**/)
> >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>> // Reduce
> >>>>> Reducer<Long> reducer;
> >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Aggregate Windowed with Custom Store
> >>>>> Initializer<String> initializer;
> >>>>> Aggregator<String, Long, String> aggregator;
> >>>>> KTable<Windowed<String>, String> aggregate =
> >>>>> groupedStream.aggregate(initializer, aggregator,
> >>>>>
> >>
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >>>>> // Cogroup SessionWindowed
> >>>>> KTable<String, String> cogrouped =
> groupedStream1.cogroup(aggregator1)
> >>>>>          .cogroup(groupedStream2, aggregator2)
> >>>>>          .aggregate(initializer, aggregator,
> >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> >>>>> sessionMerger).withQueryableStoreName("my-store"));
> >>>>>
> >>>>>
> >>>>>
> >>>>> public class Count {
> >>>>>
> >>>>>      public static class Windowed extends Count {
> >>>>>          private Windows windows;
> >>>>>      }
> >>>>>      public static class SessionWindowed extends Count {
> >>>>>          private SessionWindows sessionWindows;
> >>>>>      }
> >>>>>
> >>>>>      public static Count count();
> >>>>>      public static Windowed windowed(Windows windows);
> >>>>>      public static SessionWindowed sessionWindowed(SessionWindows
> >>>>> sessionWindows);
> >>>>>
> >>>>>      // All withXXX(...) methods.
> >>>>> }
> >>>>>
> >>>>> public class KGroupedStream {
> >>>>>      public KTable<K, Long> count(Count count);
> >>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> >> count);
> >>>>> …
> >>>>> }
> >>>>>
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> From: Guozhang Wang
> >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
> >>>>> To: dev@kafka.apache.org
> >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>>
> >>>>> I played the current proposal a bit with
> >> https://github.com/dguy/kafka/
> >>>>> tree/dsl-experiment <
> https://github.com/dguy/kafka/tree/dsl-experiment
> >>> ,
> >>>>> and here are my observations:
> >>>>>
> >>>>> 1. Personally I prefer
> >>>>>
> >>>>>      "stream.group(mapper) / stream.groupByKey()"
> >>>>>
> >>>>> than
> >>>>>
> >>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
> >>>>>
> >>>>> Since 1) withKeyMapper is not enforced programmatically though it is
> >> not
> >>>>> "really" optional like others, 2) syntax-wise it reads more natural.
> >>>>>
> >>>>> I think it is okay to add the APIs in (
> >>>>>
> >>>>>
> >>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> >>>>> )
> >>>>> in KGroupedStream.
> >>>>>
> >>>>>
> >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
> pass
> >> in
> >>>>> the most-inner state store supplier (e.g. then one whose get() return
> >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
> with
> >>>>> logging / metrics / etc? I think it would be more useful to only
> >> require
> >>>>> users pass in the inner state store supplier while specifying
> caching /
> >>>>> logging through other APIs.
> >>>>>
> >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me:
> we
> >>>> are
> >>>>> allowing users to call other APIs like "withQueryableName" multiple
> >> time,
> >>>>> but only call "withStateStoreSupplier" only once in the end. Why is
> >> that?
> >>>>>
> >>>>> 3. The current DSL seems to be only for aggregations, what about
> joins?
> >>>>>
> >>>>>
> >>>>> 4. I think it is okay to keep the "withLogConfig": for the
> >>>>> StateStoreSupplier it will still be user code specifying the topology
> >> so
> >>>> I
> >>>>> do not see there is a big difference.
> >>>>>
> >>>>>
> >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> >>>>> windowed state store supplier to enforce typing?
> >>>>>
> >>>>>
> >>>>> Below are minor ones:
> >>>>>
> >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> >>>>>
> >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >>>>>
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> >> matthias@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> I see your point about "when to add the processor to the topology".
> >>>> That
> >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
> >>>>> topology...
> >>>>>> I don't see any problem with having all the withXX() in KTable
> >>>> interface
> >>>>>> -- but this might be subjective.
> >>>>>>
> >>>>>>
> >>>>>> However, I don't understand your argument about putting aggregate()
> >>>>>> after the withXX() -- all the calls to withXX() set optional
> >> parameters
> >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> might
> >>>>>> be quite confusion for developers.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> >>>>>>>> I also think that mixing optional parameters with configs is a bad
> >>>>> idea.
> >>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
> >>>> to
> >>>>>>>> find some time to come up with something.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> Yes, i don't like the mix of config either. But the only real
> config
> >>>>> here
> >>>>>>> is the logging config - which we don't really need as it can
> already
> >>>> be
> >>>>>>> done via a custom StateStoreSupplier.
> >>>>>>>
> >>>>>>>
> >>>>>>>> What I don't like in the current proposal is the
> >>>>>>>> .grouped().withKeyMapper() -- the current solution with
> >>>> .groupBy(...)
> >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
> >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
> find
> >>>>>>>> some better names).
> >>>>>>>>
> >>>>>>>>
> >>>>>>> it could be groupByKey(), groupBy() or something different bt
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>> The proposed pattern "chains" grouping and aggregation too close
> >>>>>>>> together. I would rather separate both more than less, ie, do into
> >>>> the
> >>>>>>>> opposite direction.
> >>>>>>>>
> >>>>>>>> I am also wondering, if we could so something more "fluent". The
> >>>>> initial
> >>>>>>>> proposal was like:
> >>>>>>>>
> >>>>>>>>>> groupedStream.count()
> >>>>>>>>>>     .withStoreName("name")
> >>>>>>>>>>     .withCachingEnabled(false)
> >>>>>>>>>>     .withLoggingEnabled(config)
> >>>>>>>>>>     .table()
> >>>>>>>> The .table() statement in the end was kinda alien.
> >>>>>>>>
> >>>>>>> I agree, but then all of the withXXX methods need to be on KTable
> >>>> which
> >>>>>> is
> >>>>>>> worse in my opinion. You also need something that is going to
> "build"
> >>>>> the
> >>>>>>> internal processors and add them to the topology.
> >>>>>>>
> >>>>>>>
> >>>>>>>> The current proposal put the count() into the end -- ie, the
> >>>> optional
> >>>>>>>> parameter for count() have to specified on the .grouped() call --
> >>>> this
> >>>>>>>> does not seems to be the best way either.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> I actually prefer this method as you are building a grouped stream
> >>>> that
> >>>>>> you
> >>>>>>> will aggregate. So
> >>>> table.grouped(...).withOptionalStuff().aggregate(..)
> >>>>>> etc
> >>>>>>> seems natural to me.
> >>>>>>>
> >>>>>>>
> >>>>>>>> I did not think this through in detail, but can't we just do the
> >>>>> initial
> >>>>>>>> proposal with the .table() ?
> >>>>>>>>
> >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> >>>>>>>>
> >>>>>>>> Each .withXXX(...) return the current KTable and all the
> .withXXX()
> >>>>> are
> >>>>>>>> just added to the KTable interface. Or do I miss anything why this
> >>>>> wont'
> >>>>>>>> work or any obvious disadvantage?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>> See above.
> >>>>>>>
> >>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> >>>> fluent
> >>>>>>>>> approach, but i think it is slightly nicer.
> >>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
> >>>> the
> >>>>>> DSL,
> >>>>>>>>> but i think that enabling caching and enabling logging are things
> >>>>> that
> >>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> >>>> (even
> >>>>>>>>> though it is below) as this is actually config and we already
> have
> >>>> a
> >>>>>> way
> >>>>>>>> of
> >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
> that
> >>>>> is
> >>>>>> a
> >>>>>>>>> bit of a tedious process for someone that just wants to use the
> >>>>> default
> >>>>>>>>> storage engine, but not have caching enabled.
> >>>>>>>>>
> >>>>>>>>> There is also an orthogonal concern that Guozhang alluded to....
> If
> >>>>> you
> >>>>>>>>> want to plug in a custom storage engine and you want it to be
> >>>> logged
> >>>>>> etc,
> >>>>>>>>> you would currently need to implement that yourself. Ideally we
> can
> >>>>>>>> provide
> >>>>>>>>> a way where we will wrap the custom store with logging, metrics,
> >>>>> etc. I
> >>>>>>>>> need to think about where this fits, it is probably more
> >>>> appropriate
> >>>>> on
> >>>>>>>> the
> >>>>>>>>> Stores API.
> >>>>>>>>>
> >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>>>>>>>> // count with mapped key
> >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
> >>>>>>>>>          .withKeyMapper(keyMapper)
> >>>>>>>>>          .withKeySerde(Serdes.Long())
> >>>>>>>>>          .withValueSerde(Serdes.String())
> >>>>>>>>>          .withQueryableName("my-store")
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // windowed count
> >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> >>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("my-window-store")
> >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // windowed reduce
> >>>>>>>>> final Reducer<String> windowedReducer = null;
> >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> >>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("my-window-store")
> >>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>          .reduce(windowedReducer);
> >>>>>>>>>
> >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
> >>>>>>>>> final Initializer<Long> init = null;
> >>>>>>>>>
> >>>>>>>>> // aggregate
> >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> >>>>>>>>>          .withQueryableName("my-aggregate-store")
> >>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
> >>>>>>>>>
> >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> >>>>>> stateStoreSupplier
> >>>>>>>> = null;
> >>>>>>>>> // aggregate with custom store
> >>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
> >>>>>>>>>          .aggregate(aggregator, init);
> >>>>>>>>>
> >>>>>>>>> // disable caching
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("name")
> >>>>>>>>>          .withCachingEnabled(false)
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // disable logging
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withQueryableName("q")
> >>>>>>>>>          .withLoggingEnabled(false)
> >>>>>>>>>          .count();
> >>>>>>>>>
> >>>>>>>>> // override log config
> >>>>>>>>> final Reducer<String> reducer = null;
> >>>>>>>>> stream.grouped()
> >>>>>>>>>          .withLogConfig(Collections.singletonMap("segment.size",
> >>>>> "10"))
> >>>>>>>>>          .reduce(reducer);
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> If anyone wants to play around with this you can find the code
> >>>> here:
> >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>>>>>>>
> >>>>>>>>> Note: It won't actually work as most of the methods just return
> >>>> null.
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> >>>> wrote:
> >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
> >>>> are
> >>>>>>>> better
> >>>>>>>>>> than overload abuse.
> >>>>>>>>>>
> >>>>>>>>>> The fluent API approach reads better, no mention of builder or
> >>>> build
> >>>>>>>>>> anywhere. The main downside is that the method signatures are a
> >>>>> little
> >>>>>>>> less
> >>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
> >>>>> knows
> >>>>>>>> what
> >>>>>>>>>> it returns. Also, one needs to figure out the special method
> >>>>>> (`table()`
> >>>>>>>> in
> >>>>>>>>>> this case) that gives you what you actually care about (`KTable`
> >>>> in
> >>>>>> this
> >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
> >>>>>>>> comparison.
> >>>>>>>>>> The builder approach avoids the issues mentioned above, but it
> >>>>> doesn't
> >>>>>>>> read
> >>>>>>>>>> as well.
> >>>>>>>>>>
> >>>>>>>>>> Ismael
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
> damian.guy@gmail.com
> >>>>>>>> wrote:
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> I'd like to get a discussion going around some of the API
> choices
> >>>>>> we've
> >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
> >>>>>> operations
> >>>>>>>>>>> (though this could expand).
> >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
> >>>> i.e,
> >>>>>>>> there
> >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> >>>> noisy
> >>>>>> and
> >>>>>>>> i
> >>>>>>>>>>> feel it is only going to get worse as we add more optional
> >>>> params.
> >>>>> In
> >>>>>>>>>>> particular we've had some requests to be able to turn caching
> >>>> off,
> >>>>> or
> >>>>>>>>>>> change log configs,  on a per operator basis (note this can be
> >>>> done
> >>>>>> now
> >>>>>>>>>> if
> >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> >>>>> cumbersome).
> >>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
> >>>>>>>> overloads
> >>>>>>>>>>> so that it flows, is simple to use and understand, and is
> easily
> >>>>>>>> extended
> >>>>>>>>>>> in the future?
> >>>>>>>>>>>
> >>>>>>>>>>> One option would be to use a fluent API approach for providing
> >>>> the
> >>>>>>>>>> optional
> >>>>>>>>>>> params, so something like this:
> >>>>>>>>>>>
> >>>>>>>>>>> groupedStream.count()
> >>>>>>>>>>>     .withStoreName("name")
> >>>>>>>>>>>     .withCachingEnabled(false)
> >>>>>>>>>>>     .withLoggingEnabled(config)
> >>>>>>>>>>>     .table()
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Another option would be to provide a Builder to the count
> method,
> >>>>> so
> >>>>>> it
> >>>>>>>>>>> would look something like this:
> >>>>>>>>>>> groupedStream.count(new
> >>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>>>>>>>
> >>>>>>>>>>> Another option is to say: Hey we don't need this, what are you
> on
> >>>>>>>> about!
> >>>>>>>>>>> The above has focussed on state store related overloads, but
> the
> >>>>> same
> >>>>>>>>>> ideas
> >>>>>>>>>>> could  be applied to joins etc, where we presently have many
> join
> >>>>>>>> methods
> >>>>>>>>>>> and many overloads.
> >>>>>>>>>>>
> >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Damian
> >>>>>>>>>>>
> >>>>>>>>
> >>>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>>
> >>>>
> >>
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Damian,

I do see your point of something needs to change. But I fully agree with 
Gouzhang when he says.
---

But since this is a incompatibility change, and we are going to remove the
compatibility annotations soon it means we only have one chance and we
really have to make it right.
----

I fear all suggestions do not go far enough to become something that will carry on for very much longer.
I am currently working on KAFKA-3705 and try to find the most easy way for the user to give me all the required functionality. The easiest interface I could come up so far can be looked at here.

https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L622

And its already horribly complicated. I am currently unable to find the right abstraction level to have everything falling into place naturally. To be honest I already think introducing

https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L493

was unideal and makes everything a mess. The JoinType:Whatever is also not really flexible. 2 things come to my mind:

1. I don't think we should rule out config based decisions say configs like
	streams.$applicationID.joins.$joinname.conf = value
This can allow for tremendous changes without single API change and IMO it was not considered enough yet.

2. Push logic from the DSL to the Callback classes. A ValueJoiner for example can be used to implement different join types as the user wishes.

As Gouzhang said: stopping to break users is very important. especially with this changes + All the plans I sadly only have in my head but hopefully the first link can give a glimpse.

Thanks for preparing the examples made it way clearer to me what exactly we are talking about. I would argue to go a bit slower and more carefull on this one. At some point we need to get it right. Peeking over to the hadoop guys with their hughe userbase. Config files really work well for them.

Best Jan





On 30.06.2017 09:31, Damian Guy wrote:
> Thanks Matthias
>
> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io> wrote:
>
>> I am just catching up on this thread, so sorry for the long email in
>> advance... Also, it's to some extend a dump of thoughts and not always a
>> clear proposal. Still need to think about this in more detail. But maybe
>> it helps other to get new ideas :)
>>
>>
>>>> However, I don't understand your argument about putting aggregate()
>>>> after the withXX() -- all the calls to withXX() set optional parameters
>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>> be quite confusion for developers.
>>>>
>>>>
>>> I see what you are saying, but the grouped stream is effectively a no-op
>>> until you call one of the aggregate/count/reduce etc functions. So the
>>> optional params are ones that are applicable to any of the operations you
>>> can perform on this grouped stream. Then the final
>>> count()/reduce()/aggregate() call has any of the params that are
>>> required/specific to that function.
>>>
>> I understand your argument, but you don't share the conclusion. If we
>> need a "final/terminal" call, the better way might be
>>
>> .groupBy().count().withXX().build()
>>
>> (with a better name for build() though)
>>
>>
> The point is that all the other calls, i.e,withBlah, windowed, etc apply
> too all the aggregate functions. The terminal call being the actual type of
> aggregation you want to do. I personally find this more natural than
> groupBy().count().withBlah().build()
>
>
>>> groupedStream.count(/** non windowed count**/)
>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>
>> I like this. However, I don't see a reason to have windowed() and
>> sessionWindowed(). We should have one top-level `Windows` interface that
>> both `TimeWindows` and `SessionWindows` implement and just have a single
>> windowed() method that accepts all `Windows`. (I did not like the
>> separation of `SessionWindows` in the first place, and this seems to be
>> an opportunity to clean this up. It was hard to change when we
>> introduced session windows)
>>
> Yes - true we should look into that.
>
>
>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>> might also want to use windowBy() (instead of windowed()). Not sure how
>> important this is, but it seems to be inconsistent otherwise.
>>
>>
> Makes sense
>
>
>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
>> defining an inner/left/outer join is not an optional argument but a
>> first class concept and should have a proper representation in the API
>> (like the current methods join(), leftJoin, outerJoin()).
>>
>>
> Yep, i did originally have it as a required param and maybe that is what we
> go with. It could have a default, but maybe that is confusing.
>
>
>
>> About the two join API proposals, the second one has too much boiler
>> plate code for my taste. Also, the actual join() operator has only one
>> argument what is weird to me, as in my thinking process, the main
>> operator call, should have one parameter per mandatory argument but your
>> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
>> This is far from intuitive IMHO.
>>
>>
> This is the builder pattern, you only need one param as the builder has
> captured all of the required and optional arguments.
>
>
>> The first join proposal also seems to align better with the pattern
>> suggested for aggregations and having the same pattern for all operators
>> is important (as you stated already).
>>
>>
> This is why i offered two alternatives as i started out with. 1 is the
> builder pattern, the other is the more fluent pattern.
>
>
>>
>> Coming back to the config vs optional parameter. What about having a
>> method withConfig[s](...) that allow to put in the configuration?
>>
>>
> Sure, it is currently called withLogConfig() as that is the only thing that
> is really config.
>
>
>> This also raises the question if until() is a windows property?
>> Actually, until() seems to be a configuration parameter and thus, should
>> not not have it's own method.
>>
>>
> Hmmm, i don't agree. Until is a property of the window. It is going to be
> potentially different for every window operation you do in a streams app.
>
>
>>
>> Browsing throw your example DSL branch, I also saw this one:
>>
>>> final KTable<Windowed<String>, Long> windowed>
>>   groupedStream.counting()
>>>                   .windowed(TimeWindows.of(10L).until(10))
>>>                   .table();
>> This is an interesting idea, and it remind my on some feedback about "I
>> wanted to count a stream, but there was no count() method -- I first
>> needed to figure out, that I need to group the stream first to be able
>> to count it. It does make sense in hindsight but was not obvious in the
>> beginning". Thus, carrying out this thought, we could also do the
>> following:
>>
>> stream.count().groupedBy().windowedBy().table();
>>
>> -> Note, I use "grouped" and "windowed" instead of imperative here, as
>> it comes after the count()
>>
>> This would be more consistent than your proposal (that has grouping
>> before but windowing after count()). It might even allow us to enrich
>> the API with a some syntactic sugar like `stream.count().table()` to get
>> the overall count of all records (this would obviously not scale, but we
>> could support it -- if not now, maybe later).
>>
>>
> I guess i'd prefer
> stream.groupBy().windowBy().count()
> stream.groupBy().windowBy().reduce()
> stream.groupBy().count()
>
> As i said above, everything that happens before the final aggregate call
> can be applied to any of them. So it makes sense to me to do those things
> ahead of the final aggregate call.
>
>
>> Last about builder pattern. I am convinced that we need some "terminal"
>> operator/method that tells us when to add the processor to the topology.
>> But I don't see the need for a plain builder pattern that feels alien to
>> me (see my argument about the second join proposal). Using .stream() /
>> .table() as use in many examples might work. But maybe a more generic
>> name that we can use in all places like build() or apply() might also be
>> an option.
>>
>>
> Sure, a generic name might be ok.
>
>
>
>
>> -Matthias
>>
>>
>>
>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>> Thanks Kyle.
>>>
>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
>>> wrote:
>>>
>>>> Hi Damian,
>>>>
>>>>>>>> When trying to program in the fluent API that has been discussed
>> most
>>>> it
>>>>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
>> to
>>>>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
>> as
>>>> you
>>>>>> can today.
>>>> You said it yourself in another post that the grouped stream is
>>>> effectively a no-op until a count, reduce, or aggregate. The way I see
>> it
>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>> because
>>>> most of this fluent api would continue returning this (this being the
>>>> builder object currently being manipulated).
>>> So, if you ever store a reference to anything but KStreams and KTables
>> and
>>>> you use it in two different ways then its possible you make conflicting
>>>> withXXX() calls on the same builder.
>>>>
>>>>
>>> No necessarily true. It could return a new instance of the builder, i.e.,
>>> the builders being immutable. So if you held a reference to the builder
>> it
>>> would always be the same as it was when it was created.
>>>
>>>
>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>
>>>> I’ll admit that this shouldn’t happen but some user is going to do it
>>>> eventually…
>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes would
>>>> most likely be equivalent to the version withDeclaredSerdes. One work
>>>> around would be to always make copies of the config objects you are
>>>> building, but this approach has its own problem because now we have to
>>>> identify which configs are equivalent so we don’t create repeated
>>>> processors.
>>>>
>>>> The point of this long winded example is that we always have to be
>>>> thinking about all of the possible ways it could be misused by a user
>>>> (causing them to see hard to diagnose problems).
>>>>
>>> Exactly! That is the point of the discussion really.
>>>
>>>
>>>> In my attempt at a couple methods with builders I feel that I could
>>>> confidently say the user couldn’t really mess it up.
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>> The kGroupedStream is reusable and if they attempted to reuse the Count
>>>> for some reason it would throw an error message saying that a store
>> named
>>>> “my-store” already exists.
>>>>
>>>>
>>> Yes i agree and i think using builders is my preferred pattern.
>>>
>>> Cheers,
>>> Damian
>>>
>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> From: Damian Guy
>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>> To: dev@kafka.apache.org
>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>
>>>> Hi Kyle,
>>>>
>>>> Thanks for your input. Really appreciated.
>>>>
>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
>>>> wrote:
>>>>
>>>>> I like more of a builder pattern even though others have voiced against
>>>>> it. The reason I like it is because it makes it clear to the user that
>> a
>>>>> call to KGroupedStream#count will return a KTable not some intermediate
>>>>> class that I need to undetstand.
>>>>>
>>>> Yes, that makes sense.
>>>>
>>>>
>>>>> When trying to program in the fluent API that has been discussed most
>> it
>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>
>>>> I'd anticipate that once you have a KGroupedStream you can re-use it as
>> you
>>>> can today.
>>>>
>>>>
>>>>> This question doesn’t pop into my head at all in the builder pattern I
>>>>> assume I can reuse everything.
>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan
>> of
>>>>> the grouped.
>>>>>
>>>>> Yes, grouped() was more for demonstration and because groupBy() and
>>>> groupByKey() were taken! So i'd imagine the api would actually want to
>> be
>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>> depends
>>>> on maintaining backward compatibility.
>>>>
>>>>
>>>>> Unfortunately, the below approach would require atleast 2 (probably 3)
>>>>> overloads (one for returning a KTable and one for returning a KTable
>> with
>>>>> Windowed Key, probably would want to split windowed and sessionwindowed
>>>> for
>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>> Obviously not exhaustive but enough for you to get the picture. Count,
>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>> builder:
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Windowed Count
>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>
>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>>>>> // Session Count
>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>
>> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>
>>>> Above and below, i think i'd prefer it to be:
>>>> groupedStream.count(/** non windowed count**/)
>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>
>>>>
>>>>
>>>>
>>>>> // Reduce
>>>>> Reducer<Long> reducer;
>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Aggregate Windowed with Custom Store
>>>>> Initializer<String> initializer;
>>>>> Aggregator<String, Long, String> aggregator;
>>>>> KTable<Windowed<String>, String> aggregate =
>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>
>> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>>>>> // Cogroup SessionWindowed
>>>>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>>>>>          .cogroup(groupedStream2, aggregator2)
>>>>>          .aggregate(initializer, aggregator,
>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>
>>>>>
>>>>>
>>>>> public class Count {
>>>>>
>>>>>      public static class Windowed extends Count {
>>>>>          private Windows windows;
>>>>>      }
>>>>>      public static class SessionWindowed extends Count {
>>>>>          private SessionWindows sessionWindows;
>>>>>      }
>>>>>
>>>>>      public static Count count();
>>>>>      public static Windowed windowed(Windows windows);
>>>>>      public static SessionWindowed sessionWindowed(SessionWindows
>>>>> sessionWindows);
>>>>>
>>>>>      // All withXXX(...) methods.
>>>>> }
>>>>>
>>>>> public class KGroupedStream {
>>>>>      public KTable<K, Long> count(Count count);
>>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>> count);
>>>>> …
>>>>> }
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> From: Guozhang Wang
>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>> To: dev@kafka.apache.org
>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>
>>>>> I played the current proposal a bit with
>> https://github.com/dguy/kafka/
>>>>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment
>>> ,
>>>>> and here are my observations:
>>>>>
>>>>> 1. Personally I prefer
>>>>>
>>>>>      "stream.group(mapper) / stream.groupByKey()"
>>>>>
>>>>> than
>>>>>
>>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>
>>>>> Since 1) withKeyMapper is not enforced programmatically though it is
>> not
>>>>> "really" optional like others, 2) syntax-wise it reads more natural.
>>>>>
>>>>> I think it is okay to add the APIs in (
>>>>>
>>>>>
>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>> )
>>>>> in KGroupedStream.
>>>>>
>>>>>
>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass
>> in
>>>>> the most-inner state store supplier (e.g. then one whose get() return
>>>>> RocksDBStore), or it is supposed to return the most-outer supplier with
>>>>> logging / metrics / etc? I think it would be more useful to only
>> require
>>>>> users pass in the inner state store supplier while specifying caching /
>>>>> logging through other APIs.
>>>>>
>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
>>>> are
>>>>> allowing users to call other APIs like "withQueryableName" multiple
>> time,
>>>>> but only call "withStateStoreSupplier" only once in the end. Why is
>> that?
>>>>>
>>>>> 3. The current DSL seems to be only for aggregations, what about joins?
>>>>>
>>>>>
>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>> StateStoreSupplier it will still be user code specifying the topology
>> so
>>>> I
>>>>> do not see there is a big difference.
>>>>>
>>>>>
>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
>>>>> windowed state store supplier to enforce typing?
>>>>>
>>>>>
>>>>> Below are minor ones:
>>>>>
>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>
>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> I see your point about "when to add the processor to the topology".
>>>> That
>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>> topology...
>>>>>> I don't see any problem with having all the withXX() in KTable
>>>> interface
>>>>>> -- but this might be subjective.
>>>>>>
>>>>>>
>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>> after the withXX() -- all the calls to withXX() set optional
>> parameters
>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>>>> be quite confusion for developers.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>> I also think that mixing optional parameters with configs is a bad
>>>>> idea.
>>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
>>>> to
>>>>>>>> find some time to come up with something.
>>>>>>>>
>>>>>>>>
>>>>>>> Yes, i don't like the mix of config either. But the only real config
>>>>> here
>>>>>>> is the logging config - which we don't really need as it can already
>>>> be
>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>
>>>>>>>
>>>>>>>> What I don't like in the current proposal is the
>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>> .groupBy(...)
>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>>>>>>>> some better names).
>>>>>>>>
>>>>>>>>
>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> The proposed pattern "chains" grouping and aggregation too close
>>>>>>>> together. I would rather separate both more than less, ie, do into
>>>> the
>>>>>>>> opposite direction.
>>>>>>>>
>>>>>>>> I am also wondering, if we could so something more "fluent". The
>>>>> initial
>>>>>>>> proposal was like:
>>>>>>>>
>>>>>>>>>> groupedStream.count()
>>>>>>>>>>     .withStoreName("name")
>>>>>>>>>>     .withCachingEnabled(false)
>>>>>>>>>>     .withLoggingEnabled(config)
>>>>>>>>>>     .table()
>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>
>>>>>>> I agree, but then all of the withXXX methods need to be on KTable
>>>> which
>>>>>> is
>>>>>>> worse in my opinion. You also need something that is going to "build"
>>>>> the
>>>>>>> internal processors and add them to the topology.
>>>>>>>
>>>>>>>
>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>> optional
>>>>>>>> parameter for count() have to specified on the .grouped() call --
>>>> this
>>>>>>>> does not seems to be the best way either.
>>>>>>>>
>>>>>>>>
>>>>>>> I actually prefer this method as you are building a grouped stream
>>>> that
>>>>>> you
>>>>>>> will aggregate. So
>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>> etc
>>>>>>> seems natural to me.
>>>>>>>
>>>>>>>
>>>>>>>> I did not think this through in detail, but can't we just do the
>>>>> initial
>>>>>>>> proposal with the .table() ?
>>>>>>>>
>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>
>>>>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
>>>>> are
>>>>>>>> just added to the KTable interface. Or do I miss anything why this
>>>>> wont'
>>>>>>>> work or any obvious disadvantage?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>> See above.
>>>>>>>
>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>> fluent
>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
>>>> the
>>>>>> DSL,
>>>>>>>>> but i think that enabling caching and enabling logging are things
>>>>> that
>>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
>>>> (even
>>>>>>>>> though it is below) as this is actually config and we already have
>>>> a
>>>>>> way
>>>>>>>> of
>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
>>>>> is
>>>>>> a
>>>>>>>>> bit of a tedious process for someone that just wants to use the
>>>>> default
>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>
>>>>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
>>>>> you
>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>> logged
>>>>>> etc,
>>>>>>>>> you would currently need to implement that yourself. Ideally we can
>>>>>>>> provide
>>>>>>>>> a way where we will wrap the custom store with logging, metrics,
>>>>> etc. I
>>>>>>>>> need to think about where this fits, it is probably more
>>>> appropriate
>>>>> on
>>>>>>>> the
>>>>>>>>> Stores API.
>>>>>>>>>
>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>> // count with mapped key
>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>          .withKeyMapper(keyMapper)
>>>>>>>>>          .withKeySerde(Serdes.Long())
>>>>>>>>>          .withValueSerde(Serdes.String())
>>>>>>>>>          .withQueryableName("my-store")
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // windowed count
>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("my-window-store")
>>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // windowed reduce
>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("my-window-store")
>>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>          .reduce(windowedReducer);
>>>>>>>>>
>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>
>>>>>>>>> // aggregate
>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>          .withQueryableName("my-aggregate-store")
>>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>
>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>> stateStoreSupplier
>>>>>>>> = null;
>>>>>>>>> // aggregate with custom store
>>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>          .aggregate(aggregator, init);
>>>>>>>>>
>>>>>>>>> // disable caching
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("name")
>>>>>>>>>          .withCachingEnabled(false)
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // disable logging
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("q")
>>>>>>>>>          .withLoggingEnabled(false)
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // override log config
>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withLogConfig(Collections.singletonMap("segment.size",
>>>>> "10"))
>>>>>>>>>          .reduce(reducer);
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> If anyone wants to play around with this you can find the code
>>>> here:
>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>
>>>>>>>>> Note: It won't actually work as most of the methods just return
>>>> null.
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>> wrote:
>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
>>>> are
>>>>>>>> better
>>>>>>>>>> than overload abuse.
>>>>>>>>>>
>>>>>>>>>> The fluent API approach reads better, no mention of builder or
>>>> build
>>>>>>>>>> anywhere. The main downside is that the method signatures are a
>>>>> little
>>>>>>>> less
>>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
>>>>> knows
>>>>>>>> what
>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>> (`table()`
>>>>>>>> in
>>>>>>>>>> this case) that gives you what you actually care about (`KTable`
>>>> in
>>>>>> this
>>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>>>> comparison.
>>>>>>>>>> The builder approach avoids the issues mentioned above, but it
>>>>> doesn't
>>>>>>>> read
>>>>>>>>>> as well.
>>>>>>>>>>
>>>>>>>>>> Ismael
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>>>>>> wrote:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> I'd like to get a discussion going around some of the API choices
>>>>>> we've
>>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>>>> operations
>>>>>>>>>>> (though this could expand).
>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
>>>> i.e,
>>>>>>>> there
>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>> noisy
>>>>>> and
>>>>>>>> i
>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>> params.
>>>>> In
>>>>>>>>>>> particular we've had some requests to be able to turn caching
>>>> off,
>>>>> or
>>>>>>>>>>> change log configs,  on a per operator basis (note this can be
>>>> done
>>>>>> now
>>>>>>>>>> if
>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>> cumbersome).
>>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>>>>> overloads
>>>>>>>>>>> so that it flows, is simple to use and understand, and is easily
>>>>>>>> extended
>>>>>>>>>>> in the future?
>>>>>>>>>>>
>>>>>>>>>>> One option would be to use a fluent API approach for providing
>>>> the
>>>>>>>>>> optional
>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>
>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>     .withStoreName("name")
>>>>>>>>>>>     .withCachingEnabled(false)
>>>>>>>>>>>     .withLoggingEnabled(config)
>>>>>>>>>>>     .table()
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Another option would be to provide a Builder to the count method,
>>>>> so
>>>>>> it
>>>>>>>>>>> would look something like this:
>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>>>>>
>>>>>>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>>>>>> about!
>>>>>>>>>>> The above has focussed on state store related overloads, but the
>>>>> same
>>>>>>>>>> ideas
>>>>>>>>>>> could  be applied to joins etc, where we presently have many join
>>>>>>>> methods
>>>>>>>>>>> and many overloads.
>>>>>>>>>>>
>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>>
>>>>
>>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.

On 6/30/17 10:49 AM, Damian Guy wrote:
> On Fri, 30 Jun 2017 at 18:40 Matthias J. Sax <ma...@confluent.io> wrote:
> 
>>> Hmmm, i don't agree. Until is a property of the window. It is going to be
>>> potentially different for every window operation you do in a streams app.
>>
>> I am not sure.
>>
>> (1) for me, it is definitely a config for the operator for how long
>> windows are maintained. It's not part of the window definition (as per
>> definition there is nothing like a retention time -- that's just a
>> practical necessity as we don't have infinite memory).
>>
> 
> You could also argue that the size of the window is config.
> 

Don't think so. Window size modifies the result, ie, is part the
processing logic, while retention time is not (assuming that you
configure it large enough that no late data arrives after retention
time). Having said this, you can get the same result with different
retention time, but not with different window size.

> 
>> (2) Maybe you want different values for different windows, but than we
>> need operator level configs instead of global configs. But it's not a
>> reason to add config methods to the DSL.
>>
> 
> I don't think we should go down this path. It is pretty simple as it is and
> config driven development is not something i'd like to strive for.
> 
> 
>>
>> (3) I am actually not too sure if local configs for window retention are
>> too useful.
>> (a) If you have consecutive windows, there is some dependency anyway, as
>> a downstream window cannot exploit a larger retention time than an
>> upstream window.
>> (b) Shouldn't retention time be more or less applied to the "input
>> topics" -- the point is, how long do you want to accept late date? That
>> sound like a global question for the applications with regard to the
>> input -- not necessarily a operator/window question.
>>
>>
> Sure, within a sub-topolgy it makes sense for the window retention to be
> the same. However, different sub-toplogies may have different constraints.
> 
> 
>>
>> -Matthias
>>
>> On 6/30/17 12:31 AM, Damian Guy wrote:
>>> Thanks Matthias
>>>
>>> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>>
>>>> I am just catching up on this thread, so sorry for the long email in
>>>> advance... Also, it's to some extend a dump of thoughts and not always a
>>>> clear proposal. Still need to think about this in more detail. But maybe
>>>> it helps other to get new ideas :)
>>>>
>>>>
>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>> after the withXX() -- all the calls to withXX() set optional
>> parameters
>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>>>> be quite confusion for developers.
>>>>>>
>>>>>>
>>>>> I see what you are saying, but the grouped stream is effectively a
>> no-op
>>>>> until you call one of the aggregate/count/reduce etc functions. So the
>>>>> optional params are ones that are applicable to any of the operations
>> you
>>>>> can perform on this grouped stream. Then the final
>>>>> count()/reduce()/aggregate() call has any of the params that are
>>>>> required/specific to that function.
>>>>>
>>>>
>>>> I understand your argument, but you don't share the conclusion. If we
>>>> need a "final/terminal" call, the better way might be
>>>>
>>>> .groupBy().count().withXX().build()
>>>>
>>>> (with a better name for build() though)
>>>>
>>>>
>>> The point is that all the other calls, i.e,withBlah, windowed, etc apply
>>> too all the aggregate functions. The terminal call being the actual type
>> of
>>> aggregation you want to do. I personally find this more natural than
>>> groupBy().count().withBlah().build()
>>>
>>>
>>>>> groupedStream.count(/** non windowed count**/)
>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>
>>>>
>>>> I like this. However, I don't see a reason to have windowed() and
>>>> sessionWindowed(). We should have one top-level `Windows` interface that
>>>> both `TimeWindows` and `SessionWindows` implement and just have a single
>>>> windowed() method that accepts all `Windows`. (I did not like the
>>>> separation of `SessionWindows` in the first place, and this seems to be
>>>> an opportunity to clean this up. It was hard to change when we
>>>> introduced session windows)
>>>>
>>>
>>> Yes - true we should look into that.
>>>
>>>
>>>>
>>>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>>>> might also want to use windowBy() (instead of windowed()). Not sure how
>>>> important this is, but it seems to be inconsistent otherwise.
>>>>
>>>>
>>> Makes sense
>>>
>>>
>>>>
>>>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
>>>> defining an inner/left/outer join is not an optional argument but a
>>>> first class concept and should have a proper representation in the API
>>>> (like the current methods join(), leftJoin, outerJoin()).
>>>>
>>>>
>>> Yep, i did originally have it as a required param and maybe that is what
>> we
>>> go with. It could have a default, but maybe that is confusing.
>>>
>>>
>>>
>>>> About the two join API proposals, the second one has too much boiler
>>>> plate code for my taste. Also, the actual join() operator has only one
>>>> argument what is weird to me, as in my thinking process, the main
>>>> operator call, should have one parameter per mandatory argument but your
>>>> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
>>>> This is far from intuitive IMHO.
>>>>
>>>>
>>> This is the builder pattern, you only need one param as the builder has
>>> captured all of the required and optional arguments.
>>>
>>>
>>>> The first join proposal also seems to align better with the pattern
>>>> suggested for aggregations and having the same pattern for all operators
>>>> is important (as you stated already).
>>>>
>>>>
>>> This is why i offered two alternatives as i started out with. 1 is the
>>> builder pattern, the other is the more fluent pattern.
>>>
>>>
>>>>
>>>>
>>>> Coming back to the config vs optional parameter. What about having a
>>>> method withConfig[s](...) that allow to put in the configuration?
>>>>
>>>>
>>> Sure, it is currently called withLogConfig() as that is the only thing
>> that
>>> is really config.
>>>
>>>
>>>> This also raises the question if until() is a windows property?
>>>> Actually, until() seems to be a configuration parameter and thus, should
>>>> not not have it's own method.
>>>>
>>>>
>>> Hmmm, i don't agree. Until is a property of the window. It is going to be
>>> potentially different for every window operation you do in a streams app.
>>>
>>>
>>>>
>>>>
>>>> Browsing throw your example DSL branch, I also saw this one:
>>>>
>>>>> final KTable<Windowed<String>, Long> windowed>
>>>>  groupedStream.counting()
>>>>>                  .windowed(TimeWindows.of(10L).until(10))
>>>>>                  .table();
>>>>
>>>> This is an interesting idea, and it remind my on some feedback about "I
>>>> wanted to count a stream, but there was no count() method -- I first
>>>> needed to figure out, that I need to group the stream first to be able
>>>> to count it. It does make sense in hindsight but was not obvious in the
>>>> beginning". Thus, carrying out this thought, we could also do the
>>>> following:
>>>>
>>>> stream.count().groupedBy().windowedBy().table();
>>>>
>>>> -> Note, I use "grouped" and "windowed" instead of imperative here, as
>>>> it comes after the count()
>>>>
>>>> This would be more consistent than your proposal (that has grouping
>>>> before but windowing after count()). It might even allow us to enrich
>>>> the API with a some syntactic sugar like `stream.count().table()` to get
>>>> the overall count of all records (this would obviously not scale, but we
>>>> could support it -- if not now, maybe later).
>>>>
>>>>
>>> I guess i'd prefer
>>> stream.groupBy().windowBy().count()
>>> stream.groupBy().windowBy().reduce()
>>> stream.groupBy().count()
>>>
>>> As i said above, everything that happens before the final aggregate call
>>> can be applied to any of them. So it makes sense to me to do those things
>>> ahead of the final aggregate call.
>>>
>>>
>>>> Last about builder pattern. I am convinced that we need some "terminal"
>>>> operator/method that tells us when to add the processor to the topology.
>>>> But I don't see the need for a plain builder pattern that feels alien to
>>>> me (see my argument about the second join proposal). Using .stream() /
>>>> .table() as use in many examples might work. But maybe a more generic
>>>> name that we can use in all places like build() or apply() might also be
>>>> an option.
>>>>
>>>>
>>> Sure, a generic name might be ok.
>>>
>>>
>>>
>>>
>>>>
>>>> -Matthias
>>>>
>>>>
>>>>
>>>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>>>> Thanks Kyle.
>>>>>
>>>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi Damian,
>>>>>>
>>>>>>>>>> When trying to program in the fluent API that has been discussed
>>>> most
>>>>>> it
>>>>>>>>>> feels difficult to know when you will actually get an object you
>> can
>>>>>> reuse.
>>>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
>> legal
>>>> to
>>>>>>>>>> reuse it or does this approach expect you to call grouped each
>> time?
>>>>>>
>>>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
>>>> as
>>>>>> you
>>>>>>>> can today.
>>>>>>
>>>>>> You said it yourself in another post that the grouped stream is
>>>>>> effectively a no-op until a count, reduce, or aggregate. The way I see
>>>> it
>>>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>>>> because
>>>>>> most of this fluent api would continue returning this (this being the
>>>>>> builder object currently being manipulated).
>>>>>
>>>>> So, if you ever store a reference to anything but KStreams and KTables
>>>> and
>>>>>> you use it in two different ways then its possible you make
>> conflicting
>>>>>> withXXX() calls on the same builder.
>>>>>>
>>>>>>
>>>>> No necessarily true. It could return a new instance of the builder,
>> i.e.,
>>>>> the builders being immutable. So if you held a reference to the builder
>>>> it
>>>>> would always be the same as it was when it was created.
>>>>>
>>>>>
>>>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
>>>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>>>
>>>>>> I’ll admit that this shouldn’t happen but some user is going to do it
>>>>>> eventually…
>>>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
>> would
>>>>>> most likely be equivalent to the version withDeclaredSerdes. One work
>>>>>> around would be to always make copies of the config objects you are
>>>>>> building, but this approach has its own problem because now we have to
>>>>>> identify which configs are equivalent so we don’t create repeated
>>>>>> processors.
>>>>>>
>>>>>> The point of this long winded example is that we always have to be
>>>>>> thinking about all of the possible ways it could be misused by a user
>>>>>> (causing them to see hard to diagnose problems).
>>>>>>
>>>>>
>>>>> Exactly! That is the point of the discussion really.
>>>>>
>>>>>
>>>>>>
>>>>>> In my attempt at a couple methods with builders I feel that I could
>>>>>> confidently say the user couldn’t really mess it up.
>>>>>>> // Count
>>>>>>> KTable<String, Long> count =
>>>>>>>
>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>> The kGroupedStream is reusable and if they attempted to reuse the
>> Count
>>>>>> for some reason it would throw an error message saying that a store
>>>> named
>>>>>> “my-store” already exists.
>>>>>>
>>>>>>
>>>>> Yes i agree and i think using builders is my preferred pattern.
>>>>>
>>>>> Cheers,
>>>>> Damian
>>>>>
>>>>>
>>>>>> Thanks,
>>>>>> Kyle
>>>>>>
>>>>>> From: Damian Guy
>>>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>>>> To: dev@kafka.apache.org
>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>
>>>>>> Hi Kyle,
>>>>>>
>>>>>> Thanks for your input. Really appreciated.
>>>>>>
>>>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <winkelman.kyle@gmail.com
>>>
>>>>>> wrote:
>>>>>>
>>>>>>> I like more of a builder pattern even though others have voiced
>> against
>>>>>>> it. The reason I like it is because it makes it clear to the user
>> that
>>>> a
>>>>>>> call to KGroupedStream#count will return a KTable not some
>> intermediate
>>>>>>> class that I need to undetstand.
>>>>>>>
>>>>>>
>>>>>> Yes, that makes sense.
>>>>>>
>>>>>>
>>>>>>> When trying to program in the fluent API that has been discussed most
>>>> it
>>>>>>> feels difficult to know when you will actually get an object you can
>>>>>> reuse.
>>>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
>> to
>>>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>>>
>>>>>>
>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
>> as
>>>> you
>>>>>> can today.
>>>>>>
>>>>>>
>>>>>>> This question doesn’t pop into my head at all in the builder pattern
>> I
>>>>>>> assume I can reuse everything.
>>>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
>> fan
>>>> of
>>>>>>> the grouped.
>>>>>>>
>>>>>>> Yes, grouped() was more for demonstration and because groupBy() and
>>>>>> groupByKey() were taken! So i'd imagine the api would actually want to
>>>> be
>>>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>>>> depends
>>>>>> on maintaining backward compatibility.
>>>>>>
>>>>>>
>>>>>>> Unfortunately, the below approach would require atleast 2 (probably
>> 3)
>>>>>>> overloads (one for returning a KTable and one for returning a KTable
>>>> with
>>>>>>> Windowed Key, probably would want to split windowed and
>> sessionwindowed
>>>>>> for
>>>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>>>> Obviously not exhaustive but enough for you to get the picture.
>> Count,
>>>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>>>> builder:
>>>>>>> // Count
>>>>>>> KTable<String, Long> count =
>>>>>>>
>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>> // Windowed Count
>>>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>>>
>>>>>>
>>>>
>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>>>>>>>
>>>>>>> // Session Count
>>>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>>>
>>>>>>
>>>>
>> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>>>
>>>>>>>
>>>>>> Above and below, i think i'd prefer it to be:
>>>>>> groupedStream.count(/** non windowed count**/)
>>>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>> // Reduce
>>>>>>> Reducer<Long> reducer;
>>>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>> // Aggregate Windowed with Custom Store
>>>>>>> Initializer<String> initializer;
>>>>>>> Aggregator<String, Long, String> aggregator;
>>>>>>> KTable<Windowed<String>, String> aggregate =
>>>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>>>
>>>>>>
>>>>
>> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>>>>>>>
>>>>>>> // Cogroup SessionWindowed
>>>>>>> KTable<String, String> cogrouped =
>> groupedStream1.cogroup(aggregator1)
>>>>>>>         .cogroup(groupedStream2, aggregator2)
>>>>>>>         .aggregate(initializer, aggregator,
>>>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> public class Count {
>>>>>>>
>>>>>>>     public static class Windowed extends Count {
>>>>>>>         private Windows windows;
>>>>>>>     }
>>>>>>>     public static class SessionWindowed extends Count {
>>>>>>>         private SessionWindows sessionWindows;
>>>>>>>     }
>>>>>>>
>>>>>>>     public static Count count();
>>>>>>>     public static Windowed windowed(Windows windows);
>>>>>>>     public static SessionWindowed sessionWindowed(SessionWindows
>>>>>>> sessionWindows);
>>>>>>>
>>>>>>>     // All withXXX(...) methods.
>>>>>>> }
>>>>>>>
>>>>>>> public class KGroupedStream {
>>>>>>>     public KTable<K, Long> count(Count count);
>>>>>>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>>>>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>>>> count);
>>>>>>> …
>>>>>>> }
>>>>>>>
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Kyle
>>>>>>>
>>>>>>> From: Guozhang Wang
>>>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>>>> To: dev@kafka.apache.org
>>>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>>>
>>>>>>> I played the current proposal a bit with
>>>> https://github.com/dguy/kafka/
>>>>>>> tree/dsl-experiment <
>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>> ,
>>>>>>> and here are my observations:
>>>>>>>
>>>>>>> 1. Personally I prefer
>>>>>>>
>>>>>>>     "stream.group(mapper) / stream.groupByKey()"
>>>>>>>
>>>>>>> than
>>>>>>>
>>>>>>>     "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>>>
>>>>>>> Since 1) withKeyMapper is not enforced programmatically though it is
>>>> not
>>>>>>> "really" optional like others, 2) syntax-wise it reads more natural.
>>>>>>>
>>>>>>> I think it is okay to add the APIs in (
>>>>>>>
>>>>>>>
>>>>>>
>>>>
>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>>>> )
>>>>>>> in KGroupedStream.
>>>>>>>
>>>>>>>
>>>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
>> pass
>>>> in
>>>>>>> the most-inner state store supplier (e.g. then one whose get() return
>>>>>>> RocksDBStore), or it is supposed to return the most-outer supplier
>> with
>>>>>>> logging / metrics / etc? I think it would be more useful to only
>>>> require
>>>>>>> users pass in the inner state store supplier while specifying
>> caching /
>>>>>>> logging through other APIs.
>>>>>>>
>>>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me:
>> we
>>>>>> are
>>>>>>> allowing users to call other APIs like "withQueryableName" multiple
>>>> time,
>>>>>>> but only call "withStateStoreSupplier" only once in the end. Why is
>>>> that?
>>>>>>>
>>>>>>>
>>>>>>> 3. The current DSL seems to be only for aggregations, what about
>> joins?
>>>>>>>
>>>>>>>
>>>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>>>> StateStoreSupplier it will still be user code specifying the topology
>>>> so
>>>>>> I
>>>>>>> do not see there is a big difference.
>>>>>>>
>>>>>>>
>>>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
>>>>>>> windowed state store supplier to enforce typing?
>>>>>>>
>>>>>>>
>>>>>>> Below are minor ones:
>>>>>>>
>>>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>>>
>>>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> Guozhang
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>>>> matthias@confluent.io>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> I see your point about "when to add the processor to the topology".
>>>>>> That
>>>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>>>> topology...
>>>>>>>>
>>>>>>>> I don't see any problem with having all the withXX() in KTable
>>>>>> interface
>>>>>>>> -- but this might be subjective.
>>>>>>>>
>>>>>>>>
>>>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>>>> after the withXX() -- all the calls to withXX() set optional
>>>> parameters
>>>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
>> might
>>>>>>>> be quite confusion for developers.
>>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>>>> I also think that mixing optional parameters with configs is a bad
>>>>>>> idea.
>>>>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
>>>>>> to
>>>>>>>>>> find some time to come up with something.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> Yes, i don't like the mix of config either. But the only real
>> config
>>>>>>> here
>>>>>>>>> is the logging config - which we don't really need as it can
>> already
>>>>>> be
>>>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> What I don't like in the current proposal is the
>>>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>>>> .groupBy(...)
>>>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
>> find
>>>>>>>>>> some better names).
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> The proposed pattern "chains" grouping and aggregation too close
>>>>>>>>>> together. I would rather separate both more than less, ie, do into
>>>>>> the
>>>>>>>>>> opposite direction.
>>>>>>>>>>
>>>>>>>>>> I am also wondering, if we could so something more "fluent". The
>>>>>>> initial
>>>>>>>>>> proposal was like:
>>>>>>>>>>
>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>    .withStoreName("name")
>>>>>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>>>>>    .table()
>>>>>>>>>>
>>>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>> I agree, but then all of the withXXX methods need to be on KTable
>>>>>> which
>>>>>>>> is
>>>>>>>>> worse in my opinion. You also need something that is going to
>> "build"
>>>>>>> the
>>>>>>>>> internal processors and add them to the topology.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>>>> optional
>>>>>>>>>> parameter for count() have to specified on the .grouped() call --
>>>>>> this
>>>>>>>>>> does not seems to be the best way either.
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> I actually prefer this method as you are building a grouped stream
>>>>>> that
>>>>>>>> you
>>>>>>>>> will aggregate. So
>>>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>>>> etc
>>>>>>>>> seems natural to me.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>> I did not think this through in detail, but can't we just do the
>>>>>>> initial
>>>>>>>>>> proposal with the .table() ?
>>>>>>>>>>
>>>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>>>
>>>>>>>>>> Each .withXXX(...) return the current KTable and all the
>> .withXXX()
>>>>>>> are
>>>>>>>>>> just added to the KTable interface. Or do I miss anything why this
>>>>>>> wont'
>>>>>>>>>> work or any obvious disadvantage?
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>> See above.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> -Matthias
>>>>>>>>>>
>>>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>>>> fluent
>>>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
>>>>>> the
>>>>>>>> DSL,
>>>>>>>>>>> but i think that enabling caching and enabling logging are things
>>>>>>> that
>>>>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
>>>>>> (even
>>>>>>>>>>> though it is below) as this is actually config and we already
>> have
>>>>>> a
>>>>>>>> way
>>>>>>>>>> of
>>>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
>> that
>>>>>>> is
>>>>>>>> a
>>>>>>>>>>> bit of a tedious process for someone that just wants to use the
>>>>>>> default
>>>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>>>
>>>>>>>>>>> There is also an orthogonal concern that Guozhang alluded to....
>> If
>>>>>>> you
>>>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>>>> logged
>>>>>>>> etc,
>>>>>>>>>>> you would currently need to implement that yourself. Ideally we
>> can
>>>>>>>>>> provide
>>>>>>>>>>> a way where we will wrap the custom store with logging, metrics,
>>>>>>> etc. I
>>>>>>>>>>> need to think about where this fits, it is probably more
>>>>>> appropriate
>>>>>>> on
>>>>>>>>>> the
>>>>>>>>>>> Stores API.
>>>>>>>>>>>
>>>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>>>> // count with mapped key
>>>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>>>         .withKeyMapper(keyMapper)
>>>>>>>>>>>         .withKeySerde(Serdes.Long())
>>>>>>>>>>>         .withValueSerde(Serdes.String())
>>>>>>>>>>>         .withQueryableName("my-store")
>>>>>>>>>>>         .count();
>>>>>>>>>>>
>>>>>>>>>>> // windowed count
>>>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>>>> stream.grouped()
>>>>>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>         .count();
>>>>>>>>>>>
>>>>>>>>>>> // windowed reduce
>>>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>>>> stream.grouped()
>>>>>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>>>         .reduce(windowedReducer);
>>>>>>>>>>>
>>>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>>>
>>>>>>>>>>> // aggregate
>>>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>>>         .withQueryableName("my-aggregate-store")
>>>>>>>>>>>         .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>>>
>>>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>>>> stateStoreSupplier
>>>>>>>>>> = null;
>>>>>>>>>>>
>>>>>>>>>>> // aggregate with custom store
>>>>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>>>>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>>>         .aggregate(aggregator, init);
>>>>>>>>>>>
>>>>>>>>>>> // disable caching
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>         .withQueryableName("name")
>>>>>>>>>>>         .withCachingEnabled(false)
>>>>>>>>>>>         .count();
>>>>>>>>>>>
>>>>>>>>>>> // disable logging
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>         .withQueryableName("q")
>>>>>>>>>>>         .withLoggingEnabled(false)
>>>>>>>>>>>         .count();
>>>>>>>>>>>
>>>>>>>>>>> // override log config
>>>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>>>> stream.grouped()
>>>>>>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
>>>>>>> "10"))
>>>>>>>>>>>         .reduce(reducer);
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> If anyone wants to play around with this you can find the code
>>>>>> here:
>>>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>>>
>>>>>>>>>>> Note: It won't actually work as most of the methods just return
>>>>>> null.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>>>> wrote:
>>>>>>>>>>>
>>>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
>>>>>> are
>>>>>>>>>> better
>>>>>>>>>>>> than overload abuse.
>>>>>>>>>>>>
>>>>>>>>>>>> The fluent API approach reads better, no mention of builder or
>>>>>> build
>>>>>>>>>>>> anywhere. The main downside is that the method signatures are a
>>>>>>> little
>>>>>>>>>> less
>>>>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
>>>>>>> knows
>>>>>>>>>> what
>>>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>>>> (`table()`
>>>>>>>>>> in
>>>>>>>>>>>> this case) that gives you what you actually care about (`KTable`
>>>>>> in
>>>>>>>> this
>>>>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>>>>>> comparison.
>>>>>>>>>>>>
>>>>>>>>>>>> The builder approach avoids the issues mentioned above, but it
>>>>>>> doesn't
>>>>>>>>>> read
>>>>>>>>>>>> as well.
>>>>>>>>>>>>
>>>>>>>>>>>> Ismael
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
>> damian.guy@gmail.com
>>>>>>>
>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> Hi,
>>>>>>>>>>>>>
>>>>>>>>>>>>> I'd like to get a discussion going around some of the API
>> choices
>>>>>>>> we've
>>>>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>>>>>> operations
>>>>>>>>>>>>> (though this could expand).
>>>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
>>>>>> i.e,
>>>>>>>>>> there
>>>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>>>> noisy
>>>>>>>> and
>>>>>>>>>> i
>>>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>>>> params.
>>>>>>> In
>>>>>>>>>>>>> particular we've had some requests to be able to turn caching
>>>>>> off,
>>>>>>> or
>>>>>>>>>>>>> change log configs,  on a per operator basis (note this can be
>>>>>> done
>>>>>>>> now
>>>>>>>>>>>> if
>>>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>>>> cumbersome).
>>>>>>>>>>>>>
>>>>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>>>>>>> overloads
>>>>>>>>>>>>> so that it flows, is simple to use and understand, and is
>> easily
>>>>>>>>>> extended
>>>>>>>>>>>>> in the future?
>>>>>>>>>>>>>
>>>>>>>>>>>>> One option would be to use a fluent API approach for providing
>>>>>> the
>>>>>>>>>>>> optional
>>>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>>>
>>>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>>>    .withStoreName("name")
>>>>>>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>>>>>>    .table()
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Another option would be to provide a Builder to the count
>> method,
>>>>>>> so
>>>>>>>> it
>>>>>>>>>>>>> would look something like this:
>>>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>>>>>>>
>>>>>>>>>>>>> Another option is to say: Hey we don't need this, what are you
>> on
>>>>>>>>>> about!
>>>>>>>>>>>>>
>>>>>>>>>>>>> The above has focussed on state store related overloads, but
>> the
>>>>>>> same
>>>>>>>>>>>> ideas
>>>>>>>>>>>>> could  be applied to joins etc, where we presently have many
>> join
>>>>>>>>>> methods
>>>>>>>>>>>>> and many overloads.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>> -- Guozhang
>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
On Fri, 30 Jun 2017 at 18:40 Matthias J. Sax <ma...@confluent.io> wrote:

> > Hmmm, i don't agree. Until is a property of the window. It is going to be
> > potentially different for every window operation you do in a streams app.
>
> I am not sure.
>
> (1) for me, it is definitely a config for the operator for how long
> windows are maintained. It's not part of the window definition (as per
> definition there is nothing like a retention time -- that's just a
> practical necessity as we don't have infinite memory).
>

You could also argue that the size of the window is config.


> (2) Maybe you want different values for different windows, but than we
> need operator level configs instead of global configs. But it's not a
> reason to add config methods to the DSL.
>

I don't think we should go down this path. It is pretty simple as it is and
config driven development is not something i'd like to strive for.


>
> (3) I am actually not too sure if local configs for window retention are
> too useful.
> (a) If you have consecutive windows, there is some dependency anyway, as
> a downstream window cannot exploit a larger retention time than an
> upstream window.
> (b) Shouldn't retention time be more or less applied to the "input
> topics" -- the point is, how long do you want to accept late date? That
> sound like a global question for the applications with regard to the
> input -- not necessarily a operator/window question.
>
>
Sure, within a sub-topolgy it makes sense for the window retention to be
the same. However, different sub-toplogies may have different constraints.


>
> -Matthias
>
> On 6/30/17 12:31 AM, Damian Guy wrote:
> > Thanks Matthias
> >
> > On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io>
> wrote:
> >
> >> I am just catching up on this thread, so sorry for the long email in
> >> advance... Also, it's to some extend a dump of thoughts and not always a
> >> clear proposal. Still need to think about this in more detail. But maybe
> >> it helps other to get new ideas :)
> >>
> >>
> >>>> However, I don't understand your argument about putting aggregate()
> >>>> after the withXX() -- all the calls to withXX() set optional
> parameters
> >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >>>> be quite confusion for developers.
> >>>>
> >>>>
> >>> I see what you are saying, but the grouped stream is effectively a
> no-op
> >>> until you call one of the aggregate/count/reduce etc functions. So the
> >>> optional params are ones that are applicable to any of the operations
> you
> >>> can perform on this grouped stream. Then the final
> >>> count()/reduce()/aggregate() call has any of the params that are
> >>> required/specific to that function.
> >>>
> >>
> >> I understand your argument, but you don't share the conclusion. If we
> >> need a "final/terminal" call, the better way might be
> >>
> >> .groupBy().count().withXX().build()
> >>
> >> (with a better name for build() though)
> >>
> >>
> > The point is that all the other calls, i.e,withBlah, windowed, etc apply
> > too all the aggregate functions. The terminal call being the actual type
> of
> > aggregation you want to do. I personally find this more natural than
> > groupBy().count().withBlah().build()
> >
> >
> >>> groupedStream.count(/** non windowed count**/)
> >>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>
> >>
> >> I like this. However, I don't see a reason to have windowed() and
> >> sessionWindowed(). We should have one top-level `Windows` interface that
> >> both `TimeWindows` and `SessionWindows` implement and just have a single
> >> windowed() method that accepts all `Windows`. (I did not like the
> >> separation of `SessionWindows` in the first place, and this seems to be
> >> an opportunity to clean this up. It was hard to change when we
> >> introduced session windows)
> >>
> >
> > Yes - true we should look into that.
> >
> >
> >>
> >> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> >> might also want to use windowBy() (instead of windowed()). Not sure how
> >> important this is, but it seems to be inconsistent otherwise.
> >>
> >>
> > Makes sense
> >
> >
> >>
> >> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
> >> defining an inner/left/outer join is not an optional argument but a
> >> first class concept and should have a proper representation in the API
> >> (like the current methods join(), leftJoin, outerJoin()).
> >>
> >>
> > Yep, i did originally have it as a required param and maybe that is what
> we
> > go with. It could have a default, but maybe that is confusing.
> >
> >
> >
> >> About the two join API proposals, the second one has too much boiler
> >> plate code for my taste. Also, the actual join() operator has only one
> >> argument what is weird to me, as in my thinking process, the main
> >> operator call, should have one parameter per mandatory argument but your
> >> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
> >> This is far from intuitive IMHO.
> >>
> >>
> > This is the builder pattern, you only need one param as the builder has
> > captured all of the required and optional arguments.
> >
> >
> >> The first join proposal also seems to align better with the pattern
> >> suggested for aggregations and having the same pattern for all operators
> >> is important (as you stated already).
> >>
> >>
> > This is why i offered two alternatives as i started out with. 1 is the
> > builder pattern, the other is the more fluent pattern.
> >
> >
> >>
> >>
> >> Coming back to the config vs optional parameter. What about having a
> >> method withConfig[s](...) that allow to put in the configuration?
> >>
> >>
> > Sure, it is currently called withLogConfig() as that is the only thing
> that
> > is really config.
> >
> >
> >> This also raises the question if until() is a windows property?
> >> Actually, until() seems to be a configuration parameter and thus, should
> >> not not have it's own method.
> >>
> >>
> > Hmmm, i don't agree. Until is a property of the window. It is going to be
> > potentially different for every window operation you do in a streams app.
> >
> >
> >>
> >>
> >> Browsing throw your example DSL branch, I also saw this one:
> >>
> >>> final KTable<Windowed<String>, Long> windowed>
> >>  groupedStream.counting()
> >>>                  .windowed(TimeWindows.of(10L).until(10))
> >>>                  .table();
> >>
> >> This is an interesting idea, and it remind my on some feedback about "I
> >> wanted to count a stream, but there was no count() method -- I first
> >> needed to figure out, that I need to group the stream first to be able
> >> to count it. It does make sense in hindsight but was not obvious in the
> >> beginning". Thus, carrying out this thought, we could also do the
> >> following:
> >>
> >> stream.count().groupedBy().windowedBy().table();
> >>
> >> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> >> it comes after the count()
> >>
> >> This would be more consistent than your proposal (that has grouping
> >> before but windowing after count()). It might even allow us to enrich
> >> the API with a some syntactic sugar like `stream.count().table()` to get
> >> the overall count of all records (this would obviously not scale, but we
> >> could support it -- if not now, maybe later).
> >>
> >>
> > I guess i'd prefer
> > stream.groupBy().windowBy().count()
> > stream.groupBy().windowBy().reduce()
> > stream.groupBy().count()
> >
> > As i said above, everything that happens before the final aggregate call
> > can be applied to any of them. So it makes sense to me to do those things
> > ahead of the final aggregate call.
> >
> >
> >> Last about builder pattern. I am convinced that we need some "terminal"
> >> operator/method that tells us when to add the processor to the topology.
> >> But I don't see the need for a plain builder pattern that feels alien to
> >> me (see my argument about the second join proposal). Using .stream() /
> >> .table() as use in many examples might work. But maybe a more generic
> >> name that we can use in all places like build() or apply() might also be
> >> an option.
> >>
> >>
> > Sure, a generic name might be ok.
> >
> >
> >
> >
> >>
> >> -Matthias
> >>
> >>
> >>
> >> On 6/29/17 7:37 AM, Damian Guy wrote:
> >>> Thanks Kyle.
> >>>
> >>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> >>> wrote:
> >>>
> >>>> Hi Damian,
> >>>>
> >>>>>>>> When trying to program in the fluent API that has been discussed
> >> most
> >>>> it
> >>>>>>>> feels difficult to know when you will actually get an object you
> can
> >>>> reuse.
> >>>>>>>> What if I make one KGroupedStream that I want to reuse, is it
> legal
> >> to
> >>>>>>>> reuse it or does this approach expect you to call grouped each
> time?
> >>>>
> >>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> >> as
> >>>> you
> >>>>>> can today.
> >>>>
> >>>> You said it yourself in another post that the grouped stream is
> >>>> effectively a no-op until a count, reduce, or aggregate. The way I see
> >> it
> >>>> you wouldn’t be able to reuse anything except KStreams and KTables,
> >> because
> >>>> most of this fluent api would continue returning this (this being the
> >>>> builder object currently being manipulated).
> >>>
> >>> So, if you ever store a reference to anything but KStreams and KTables
> >> and
> >>>> you use it in two different ways then its possible you make
> conflicting
> >>>> withXXX() calls on the same builder.
> >>>>
> >>>>
> >>> No necessarily true. It could return a new instance of the builder,
> i.e.,
> >>> the builders being immutable. So if you held a reference to the builder
> >> it
> >>> would always be the same as it was when it was created.
> >>>
> >>>
> >>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> >>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> >>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >>>>
> >>>> I’ll admit that this shouldn’t happen but some user is going to do it
> >>>> eventually…
> >>>> Depending on implementation uses of groupedStreamWithDefaultSerdes
> would
> >>>> most likely be equivalent to the version withDeclaredSerdes. One work
> >>>> around would be to always make copies of the config objects you are
> >>>> building, but this approach has its own problem because now we have to
> >>>> identify which configs are equivalent so we don’t create repeated
> >>>> processors.
> >>>>
> >>>> The point of this long winded example is that we always have to be
> >>>> thinking about all of the possible ways it could be misused by a user
> >>>> (causing them to see hard to diagnose problems).
> >>>>
> >>>
> >>> Exactly! That is the point of the discussion really.
> >>>
> >>>
> >>>>
> >>>> In my attempt at a couple methods with builders I feel that I could
> >>>> confidently say the user couldn’t really mess it up.
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>> The kGroupedStream is reusable and if they attempted to reuse the
> Count
> >>>> for some reason it would throw an error message saying that a store
> >> named
> >>>> “my-store” already exists.
> >>>>
> >>>>
> >>> Yes i agree and i think using builders is my preferred pattern.
> >>>
> >>> Cheers,
> >>> Damian
> >>>
> >>>
> >>>> Thanks,
> >>>> Kyle
> >>>>
> >>>> From: Damian Guy
> >>>> Sent: Thursday, June 29, 2017 3:59 AM
> >>>> To: dev@kafka.apache.org
> >>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>
> >>>> Hi Kyle,
> >>>>
> >>>> Thanks for your input. Really appreciated.
> >>>>
> >>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <winkelman.kyle@gmail.com
> >
> >>>> wrote:
> >>>>
> >>>>> I like more of a builder pattern even though others have voiced
> against
> >>>>> it. The reason I like it is because it makes it clear to the user
> that
> >> a
> >>>>> call to KGroupedStream#count will return a KTable not some
> intermediate
> >>>>> class that I need to undetstand.
> >>>>>
> >>>>
> >>>> Yes, that makes sense.
> >>>>
> >>>>
> >>>>> When trying to program in the fluent API that has been discussed most
> >> it
> >>>>> feels difficult to know when you will actually get an object you can
> >>>> reuse.
> >>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> to
> >>>>> reuse it or does this approach expect you to call grouped each time?
> >>>>
> >>>>
> >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> as
> >> you
> >>>> can today.
> >>>>
> >>>>
> >>>>> This question doesn’t pop into my head at all in the builder pattern
> I
> >>>>> assume I can reuse everything.
> >>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big
> fan
> >> of
> >>>>> the grouped.
> >>>>>
> >>>>> Yes, grouped() was more for demonstration and because groupBy() and
> >>>> groupByKey() were taken! So i'd imagine the api would actually want to
> >> be
> >>>> groupByKey(/** no required args***/).withOptionalArg() and
> >>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> >> depends
> >>>> on maintaining backward compatibility.
> >>>>
> >>>>
> >>>>> Unfortunately, the below approach would require atleast 2 (probably
> 3)
> >>>>> overloads (one for returning a KTable and one for returning a KTable
> >> with
> >>>>> Windowed Key, probably would want to split windowed and
> sessionwindowed
> >>>> for
> >>>>> ease of implementation) of each count, reduce, and aggregate.
> >>>>> Obviously not exhaustive but enough for you to get the picture.
> Count,
> >>>>> Reduce, and Aggregate supply 3 static methods to initialize the
> >> builder:
> >>>>> // Count
> >>>>> KTable<String, Long> count =
> >>>>>
> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Windowed Count
> >>>>> KTable<Windowed<String>, Long> windowedCount =
> >>>>>
> >>>>
> >>
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >>>>>
> >>>>> // Session Count
> >>>>> KTable<Windowed<String>, Long> sessionCount =
> >>>>>
> >>>>
> >>
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >>>>>
> >>>>>
> >>>> Above and below, i think i'd prefer it to be:
> >>>> groupedStream.count(/** non windowed count**/)
> >>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>>>
> >>>>
> >>>>
> >>>>
> >>>>> // Reduce
> >>>>> Reducer<Long> reducer;
> >>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> >>>>> Reduce.reduce().withQueryableStoreName("my-store"));
> >>>>>
> >>>>> // Aggregate Windowed with Custom Store
> >>>>> Initializer<String> initializer;
> >>>>> Aggregator<String, Long, String> aggregator;
> >>>>> KTable<Windowed<String>, String> aggregate =
> >>>>> groupedStream.aggregate(initializer, aggregator,
> >>>>>
> >>>>
> >>
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >>>>>
> >>>>> // Cogroup SessionWindowed
> >>>>> KTable<String, String> cogrouped =
> groupedStream1.cogroup(aggregator1)
> >>>>>         .cogroup(groupedStream2, aggregator2)
> >>>>>         .aggregate(initializer, aggregator,
> >>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> >>>>> sessionMerger).withQueryableStoreName("my-store"));
> >>>>>
> >>>>>
> >>>>>
> >>>>> public class Count {
> >>>>>
> >>>>>     public static class Windowed extends Count {
> >>>>>         private Windows windows;
> >>>>>     }
> >>>>>     public static class SessionWindowed extends Count {
> >>>>>         private SessionWindows sessionWindows;
> >>>>>     }
> >>>>>
> >>>>>     public static Count count();
> >>>>>     public static Windowed windowed(Windows windows);
> >>>>>     public static SessionWindowed sessionWindowed(SessionWindows
> >>>>> sessionWindows);
> >>>>>
> >>>>>     // All withXXX(...) methods.
> >>>>> }
> >>>>>
> >>>>> public class KGroupedStream {
> >>>>>     public KTable<K, Long> count(Count count);
> >>>>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >>>>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> >> count);
> >>>>> …
> >>>>> }
> >>>>>
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> From: Guozhang Wang
> >>>>> Sent: Wednesday, June 28, 2017 7:45 PM
> >>>>> To: dev@kafka.apache.org
> >>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>>>
> >>>>> I played the current proposal a bit with
> >> https://github.com/dguy/kafka/
> >>>>> tree/dsl-experiment <
> https://github.com/dguy/kafka/tree/dsl-experiment
> >>> ,
> >>>>> and here are my observations:
> >>>>>
> >>>>> 1. Personally I prefer
> >>>>>
> >>>>>     "stream.group(mapper) / stream.groupByKey()"
> >>>>>
> >>>>> than
> >>>>>
> >>>>>     "stream.group().withKeyMapper(mapper) / stream.group()"
> >>>>>
> >>>>> Since 1) withKeyMapper is not enforced programmatically though it is
> >> not
> >>>>> "really" optional like others, 2) syntax-wise it reads more natural.
> >>>>>
> >>>>> I think it is okay to add the APIs in (
> >>>>>
> >>>>>
> >>>>
> >>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> >>>>> )
> >>>>> in KGroupedStream.
> >>>>>
> >>>>>
> >>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to
> pass
> >> in
> >>>>> the most-inner state store supplier (e.g. then one whose get() return
> >>>>> RocksDBStore), or it is supposed to return the most-outer supplier
> with
> >>>>> logging / metrics / etc? I think it would be more useful to only
> >> require
> >>>>> users pass in the inner state store supplier while specifying
> caching /
> >>>>> logging through other APIs.
> >>>>>
> >>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me:
> we
> >>>> are
> >>>>> allowing users to call other APIs like "withQueryableName" multiple
> >> time,
> >>>>> but only call "withStateStoreSupplier" only once in the end. Why is
> >> that?
> >>>>>
> >>>>>
> >>>>> 3. The current DSL seems to be only for aggregations, what about
> joins?
> >>>>>
> >>>>>
> >>>>> 4. I think it is okay to keep the "withLogConfig": for the
> >>>>> StateStoreSupplier it will still be user code specifying the topology
> >> so
> >>>> I
> >>>>> do not see there is a big difference.
> >>>>>
> >>>>>
> >>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> >>>>> windowed state store supplier to enforce typing?
> >>>>>
> >>>>>
> >>>>> Below are minor ones:
> >>>>>
> >>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> >>>>>
> >>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >>>>>
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>>
> >>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> >> matthias@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>> I see your point about "when to add the processor to the topology".
> >>>> That
> >>>>>> is indeed an issue. Not sure it we could allow "updates" to the
> >>>>> topology...
> >>>>>>
> >>>>>> I don't see any problem with having all the withXX() in KTable
> >>>> interface
> >>>>>> -- but this might be subjective.
> >>>>>>
> >>>>>>
> >>>>>> However, I don't understand your argument about putting aggregate()
> >>>>>> after the withXX() -- all the calls to withXX() set optional
> >> parameters
> >>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this
> might
> >>>>>> be quite confusion for developers.
> >>>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> >>>>>>>> I also think that mixing optional parameters with configs is a bad
> >>>>> idea.
> >>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
> >>>> to
> >>>>>>>> find some time to come up with something.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> Yes, i don't like the mix of config either. But the only real
> config
> >>>>> here
> >>>>>>> is the logging config - which we don't really need as it can
> already
> >>>> be
> >>>>>>> done via a custom StateStoreSupplier.
> >>>>>>>
> >>>>>>>
> >>>>>>>> What I don't like in the current proposal is the
> >>>>>>>> .grouped().withKeyMapper() -- the current solution with
> >>>> .groupBy(...)
> >>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
> >>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should
> find
> >>>>>>>> some better names).
> >>>>>>>>
> >>>>>>>>
> >>>>>>> it could be groupByKey(), groupBy() or something different bt
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>> The proposed pattern "chains" grouping and aggregation too close
> >>>>>>>> together. I would rather separate both more than less, ie, do into
> >>>> the
> >>>>>>>> opposite direction.
> >>>>>>>>
> >>>>>>>> I am also wondering, if we could so something more "fluent". The
> >>>>> initial
> >>>>>>>> proposal was like:
> >>>>>>>>
> >>>>>>>>>> groupedStream.count()
> >>>>>>>>>>    .withStoreName("name")
> >>>>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>>>    .table()
> >>>>>>>>
> >>>>>>>> The .table() statement in the end was kinda alien.
> >>>>>>>>
> >>>>>>>
> >>>>>>> I agree, but then all of the withXXX methods need to be on KTable
> >>>> which
> >>>>>> is
> >>>>>>> worse in my opinion. You also need something that is going to
> "build"
> >>>>> the
> >>>>>>> internal processors and add them to the topology.
> >>>>>>>
> >>>>>>>
> >>>>>>>> The current proposal put the count() into the end -- ie, the
> >>>> optional
> >>>>>>>> parameter for count() have to specified on the .grouped() call --
> >>>> this
> >>>>>>>> does not seems to be the best way either.
> >>>>>>>>
> >>>>>>>>
> >>>>>>> I actually prefer this method as you are building a grouped stream
> >>>> that
> >>>>>> you
> >>>>>>> will aggregate. So
> >>>> table.grouped(...).withOptionalStuff().aggregate(..)
> >>>>>> etc
> >>>>>>> seems natural to me.
> >>>>>>>
> >>>>>>>
> >>>>>>>> I did not think this through in detail, but can't we just do the
> >>>>> initial
> >>>>>>>> proposal with the .table() ?
> >>>>>>>>
> >>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> >>>>>>>>
> >>>>>>>> Each .withXXX(...) return the current KTable and all the
> .withXXX()
> >>>>> are
> >>>>>>>> just added to the KTable interface. Or do I miss anything why this
> >>>>> wont'
> >>>>>>>> work or any obvious disadvantage?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>> See above.
> >>>>>>>
> >>>>>>>
> >>>>>>>>
> >>>>>>>> -Matthias
> >>>>>>>>
> >>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> >>>> fluent
> >>>>>>>>> approach, but i think it is slightly nicer.
> >>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
> >>>> the
> >>>>>> DSL,
> >>>>>>>>> but i think that enabling caching and enabling logging are things
> >>>>> that
> >>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> >>>> (even
> >>>>>>>>> though it is below) as this is actually config and we already
> have
> >>>> a
> >>>>>> way
> >>>>>>>> of
> >>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands
> that
> >>>>> is
> >>>>>> a
> >>>>>>>>> bit of a tedious process for someone that just wants to use the
> >>>>> default
> >>>>>>>>> storage engine, but not have caching enabled.
> >>>>>>>>>
> >>>>>>>>> There is also an orthogonal concern that Guozhang alluded to....
> If
> >>>>> you
> >>>>>>>>> want to plug in a custom storage engine and you want it to be
> >>>> logged
> >>>>>> etc,
> >>>>>>>>> you would currently need to implement that yourself. Ideally we
> can
> >>>>>>>> provide
> >>>>>>>>> a way where we will wrap the custom store with logging, metrics,
> >>>>> etc. I
> >>>>>>>>> need to think about where this fits, it is probably more
> >>>> appropriate
> >>>>> on
> >>>>>>>> the
> >>>>>>>>> Stores API.
> >>>>>>>>>
> >>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>>>>>>>> // count with mapped key
> >>>>>>>>> final KTable<Long, Long> count = stream.grouped()
> >>>>>>>>>         .withKeyMapper(keyMapper)
> >>>>>>>>>         .withKeySerde(Serdes.Long())
> >>>>>>>>>         .withValueSerde(Serdes.String())
> >>>>>>>>>         .withQueryableName("my-store")
> >>>>>>>>>         .count();
> >>>>>>>>>
> >>>>>>>>> // windowed count
> >>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> >>>> stream.grouped()
> >>>>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>         .count();
> >>>>>>>>>
> >>>>>>>>> // windowed reduce
> >>>>>>>>> final Reducer<String> windowedReducer = null;
> >>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> >>>>>> stream.grouped()
> >>>>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>>>         .reduce(windowedReducer);
> >>>>>>>>>
> >>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
> >>>>>>>>> final Initializer<Long> init = null;
> >>>>>>>>>
> >>>>>>>>> // aggregate
> >>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> >>>>>>>>>         .withQueryableName("my-aggregate-store")
> >>>>>>>>>         .aggregate(aggregator, init, Serdes.Long());
> >>>>>>>>>
> >>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> >>>>>> stateStoreSupplier
> >>>>>>>> = null;
> >>>>>>>>>
> >>>>>>>>> // aggregate with custom store
> >>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>>>>>>>         .aggregate(aggregator, init);
> >>>>>>>>>
> >>>>>>>>> // disable caching
> >>>>>>>>> stream.grouped()
> >>>>>>>>>         .withQueryableName("name")
> >>>>>>>>>         .withCachingEnabled(false)
> >>>>>>>>>         .count();
> >>>>>>>>>
> >>>>>>>>> // disable logging
> >>>>>>>>> stream.grouped()
> >>>>>>>>>         .withQueryableName("q")
> >>>>>>>>>         .withLoggingEnabled(false)
> >>>>>>>>>         .count();
> >>>>>>>>>
> >>>>>>>>> // override log config
> >>>>>>>>> final Reducer<String> reducer = null;
> >>>>>>>>> stream.grouped()
> >>>>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
> >>>>> "10"))
> >>>>>>>>>         .reduce(reducer);
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> If anyone wants to play around with this you can find the code
> >>>> here:
> >>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>>>>>>>
> >>>>>>>>> Note: It won't actually work as most of the methods just return
> >>>> null.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> >>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
> >>>> are
> >>>>>>>> better
> >>>>>>>>>> than overload abuse.
> >>>>>>>>>>
> >>>>>>>>>> The fluent API approach reads better, no mention of builder or
> >>>> build
> >>>>>>>>>> anywhere. The main downside is that the method signatures are a
> >>>>> little
> >>>>>>>> less
> >>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
> >>>>> knows
> >>>>>>>> what
> >>>>>>>>>> it returns. Also, one needs to figure out the special method
> >>>>>> (`table()`
> >>>>>>>> in
> >>>>>>>>>> this case) that gives you what you actually care about (`KTable`
> >>>> in
> >>>>>> this
> >>>>>>>>>> case). Not major issues, but worth mentioning while doing the
> >>>>>>>> comparison.
> >>>>>>>>>>
> >>>>>>>>>> The builder approach avoids the issues mentioned above, but it
> >>>>> doesn't
> >>>>>>>> read
> >>>>>>>>>> as well.
> >>>>>>>>>>
> >>>>>>>>>> Ismael
> >>>>>>>>>>
> >>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <
> damian.guy@gmail.com
> >>>>>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Hi,
> >>>>>>>>>>>
> >>>>>>>>>>> I'd like to get a discussion going around some of the API
> choices
> >>>>>> we've
> >>>>>>>>>>> made in the DLS. In particular those that relate to stateful
> >>>>>> operations
> >>>>>>>>>>> (though this could expand).
> >>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
> >>>> i.e,
> >>>>>>>> there
> >>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> >>>> noisy
> >>>>>> and
> >>>>>>>> i
> >>>>>>>>>>> feel it is only going to get worse as we add more optional
> >>>> params.
> >>>>> In
> >>>>>>>>>>> particular we've had some requests to be able to turn caching
> >>>> off,
> >>>>> or
> >>>>>>>>>>> change log configs,  on a per operator basis (note this can be
> >>>> done
> >>>>>> now
> >>>>>>>>>> if
> >>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> >>>>> cumbersome).
> >>>>>>>>>>>
> >>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
> >>>>>>>> overloads
> >>>>>>>>>>> so that it flows, is simple to use and understand, and is
> easily
> >>>>>>>> extended
> >>>>>>>>>>> in the future?
> >>>>>>>>>>>
> >>>>>>>>>>> One option would be to use a fluent API approach for providing
> >>>> the
> >>>>>>>>>> optional
> >>>>>>>>>>> params, so something like this:
> >>>>>>>>>>>
> >>>>>>>>>>> groupedStream.count()
> >>>>>>>>>>>    .withStoreName("name")
> >>>>>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>>>>    .table()
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>> Another option would be to provide a Builder to the count
> method,
> >>>>> so
> >>>>>> it
> >>>>>>>>>>> would look something like this:
> >>>>>>>>>>> groupedStream.count(new
> >>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>>>>>>>
> >>>>>>>>>>> Another option is to say: Hey we don't need this, what are you
> on
> >>>>>>>> about!
> >>>>>>>>>>>
> >>>>>>>>>>> The above has focussed on state store related overloads, but
> the
> >>>>> same
> >>>>>>>>>> ideas
> >>>>>>>>>>> could  be applied to joins etc, where we presently have many
> join
> >>>>>>>> methods
> >>>>>>>>>>> and many overloads.
> >>>>>>>>>>>
> >>>>>>>>>>> Anyway, i look forward to hearing your opinions.
> >>>>>>>>>>>
> >>>>>>>>>>> Thanks,
> >>>>>>>>>>> Damian
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>
> >>
> >
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
> Hmmm, i don't agree. Until is a property of the window. It is going to be
> potentially different for every window operation you do in a streams app.

I am not sure.

(1) for me, it is definitely a config for the operator for how long
windows are maintained. It's not part of the window definition (as per
definition there is nothing like a retention time -- that's just a
practical necessity as we don't have infinite memory).

(2) Maybe you want different values for different windows, but than we
need operator level configs instead of global configs. But it's not a
reason to add config methods to the DSL.

(3) I am actually not too sure if local configs for window retention are
too useful.
(a) If you have consecutive windows, there is some dependency anyway, as
a downstream window cannot exploit a larger retention time than an
upstream window.
(b) Shouldn't retention time be more or less applied to the "input
topics" -- the point is, how long do you want to accept late date? That
sound like a global question for the applications with regard to the
input -- not necessarily a operator/window question.


-Matthias

On 6/30/17 12:31 AM, Damian Guy wrote:
> Thanks Matthias
> 
> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io> wrote:
> 
>> I am just catching up on this thread, so sorry for the long email in
>> advance... Also, it's to some extend a dump of thoughts and not always a
>> clear proposal. Still need to think about this in more detail. But maybe
>> it helps other to get new ideas :)
>>
>>
>>>> However, I don't understand your argument about putting aggregate()
>>>> after the withXX() -- all the calls to withXX() set optional parameters
>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>> be quite confusion for developers.
>>>>
>>>>
>>> I see what you are saying, but the grouped stream is effectively a no-op
>>> until you call one of the aggregate/count/reduce etc functions. So the
>>> optional params are ones that are applicable to any of the operations you
>>> can perform on this grouped stream. Then the final
>>> count()/reduce()/aggregate() call has any of the params that are
>>> required/specific to that function.
>>>
>>
>> I understand your argument, but you don't share the conclusion. If we
>> need a "final/terminal" call, the better way might be
>>
>> .groupBy().count().withXX().build()
>>
>> (with a better name for build() though)
>>
>>
> The point is that all the other calls, i.e,withBlah, windowed, etc apply
> too all the aggregate functions. The terminal call being the actual type of
> aggregation you want to do. I personally find this more natural than
> groupBy().count().withBlah().build()
> 
> 
>>> groupedStream.count(/** non windowed count**/)
>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>
>>
>> I like this. However, I don't see a reason to have windowed() and
>> sessionWindowed(). We should have one top-level `Windows` interface that
>> both `TimeWindows` and `SessionWindows` implement and just have a single
>> windowed() method that accepts all `Windows`. (I did not like the
>> separation of `SessionWindows` in the first place, and this seems to be
>> an opportunity to clean this up. It was hard to change when we
>> introduced session windows)
>>
> 
> Yes - true we should look into that.
> 
> 
>>
>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>> might also want to use windowBy() (instead of windowed()). Not sure how
>> important this is, but it seems to be inconsistent otherwise.
>>
>>
> Makes sense
> 
> 
>>
>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
>> defining an inner/left/outer join is not an optional argument but a
>> first class concept and should have a proper representation in the API
>> (like the current methods join(), leftJoin, outerJoin()).
>>
>>
> Yep, i did originally have it as a required param and maybe that is what we
> go with. It could have a default, but maybe that is confusing.
> 
> 
> 
>> About the two join API proposals, the second one has too much boiler
>> plate code for my taste. Also, the actual join() operator has only one
>> argument what is weird to me, as in my thinking process, the main
>> operator call, should have one parameter per mandatory argument but your
>> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
>> This is far from intuitive IMHO.
>>
>>
> This is the builder pattern, you only need one param as the builder has
> captured all of the required and optional arguments.
> 
> 
>> The first join proposal also seems to align better with the pattern
>> suggested for aggregations and having the same pattern for all operators
>> is important (as you stated already).
>>
>>
> This is why i offered two alternatives as i started out with. 1 is the
> builder pattern, the other is the more fluent pattern.
> 
> 
>>
>>
>> Coming back to the config vs optional parameter. What about having a
>> method withConfig[s](...) that allow to put in the configuration?
>>
>>
> Sure, it is currently called withLogConfig() as that is the only thing that
> is really config.
> 
> 
>> This also raises the question if until() is a windows property?
>> Actually, until() seems to be a configuration parameter and thus, should
>> not not have it's own method.
>>
>>
> Hmmm, i don't agree. Until is a property of the window. It is going to be
> potentially different for every window operation you do in a streams app.
> 
> 
>>
>>
>> Browsing throw your example DSL branch, I also saw this one:
>>
>>> final KTable<Windowed<String>, Long> windowed>
>>  groupedStream.counting()
>>>                  .windowed(TimeWindows.of(10L).until(10))
>>>                  .table();
>>
>> This is an interesting idea, and it remind my on some feedback about "I
>> wanted to count a stream, but there was no count() method -- I first
>> needed to figure out, that I need to group the stream first to be able
>> to count it. It does make sense in hindsight but was not obvious in the
>> beginning". Thus, carrying out this thought, we could also do the
>> following:
>>
>> stream.count().groupedBy().windowedBy().table();
>>
>> -> Note, I use "grouped" and "windowed" instead of imperative here, as
>> it comes after the count()
>>
>> This would be more consistent than your proposal (that has grouping
>> before but windowing after count()). It might even allow us to enrich
>> the API with a some syntactic sugar like `stream.count().table()` to get
>> the overall count of all records (this would obviously not scale, but we
>> could support it -- if not now, maybe later).
>>
>>
> I guess i'd prefer
> stream.groupBy().windowBy().count()
> stream.groupBy().windowBy().reduce()
> stream.groupBy().count()
> 
> As i said above, everything that happens before the final aggregate call
> can be applied to any of them. So it makes sense to me to do those things
> ahead of the final aggregate call.
> 
> 
>> Last about builder pattern. I am convinced that we need some "terminal"
>> operator/method that tells us when to add the processor to the topology.
>> But I don't see the need for a plain builder pattern that feels alien to
>> me (see my argument about the second join proposal). Using .stream() /
>> .table() as use in many examples might work. But maybe a more generic
>> name that we can use in all places like build() or apply() might also be
>> an option.
>>
>>
> Sure, a generic name might be ok.
> 
> 
> 
> 
>>
>> -Matthias
>>
>>
>>
>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>> Thanks Kyle.
>>>
>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
>>> wrote:
>>>
>>>> Hi Damian,
>>>>
>>>>>>>> When trying to program in the fluent API that has been discussed
>> most
>>>> it
>>>>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
>> to
>>>>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>
>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
>> as
>>>> you
>>>>>> can today.
>>>>
>>>> You said it yourself in another post that the grouped stream is
>>>> effectively a no-op until a count, reduce, or aggregate. The way I see
>> it
>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>> because
>>>> most of this fluent api would continue returning this (this being the
>>>> builder object currently being manipulated).
>>>
>>> So, if you ever store a reference to anything but KStreams and KTables
>> and
>>>> you use it in two different ways then its possible you make conflicting
>>>> withXXX() calls on the same builder.
>>>>
>>>>
>>> No necessarily true. It could return a new instance of the builder, i.e.,
>>> the builders being immutable. So if you held a reference to the builder
>> it
>>> would always be the same as it was when it was created.
>>>
>>>
>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>
>>>> I’ll admit that this shouldn’t happen but some user is going to do it
>>>> eventually…
>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes would
>>>> most likely be equivalent to the version withDeclaredSerdes. One work
>>>> around would be to always make copies of the config objects you are
>>>> building, but this approach has its own problem because now we have to
>>>> identify which configs are equivalent so we don’t create repeated
>>>> processors.
>>>>
>>>> The point of this long winded example is that we always have to be
>>>> thinking about all of the possible ways it could be misused by a user
>>>> (causing them to see hard to diagnose problems).
>>>>
>>>
>>> Exactly! That is the point of the discussion really.
>>>
>>>
>>>>
>>>> In my attempt at a couple methods with builders I feel that I could
>>>> confidently say the user couldn’t really mess it up.
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>> The kGroupedStream is reusable and if they attempted to reuse the Count
>>>> for some reason it would throw an error message saying that a store
>> named
>>>> “my-store” already exists.
>>>>
>>>>
>>> Yes i agree and i think using builders is my preferred pattern.
>>>
>>> Cheers,
>>> Damian
>>>
>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> From: Damian Guy
>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>> To: dev@kafka.apache.org
>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>
>>>> Hi Kyle,
>>>>
>>>> Thanks for your input. Really appreciated.
>>>>
>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
>>>> wrote:
>>>>
>>>>> I like more of a builder pattern even though others have voiced against
>>>>> it. The reason I like it is because it makes it clear to the user that
>> a
>>>>> call to KGroupedStream#count will return a KTable not some intermediate
>>>>> class that I need to undetstand.
>>>>>
>>>>
>>>> Yes, that makes sense.
>>>>
>>>>
>>>>> When trying to program in the fluent API that has been discussed most
>> it
>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>
>>>>
>>>> I'd anticipate that once you have a KGroupedStream you can re-use it as
>> you
>>>> can today.
>>>>
>>>>
>>>>> This question doesn’t pop into my head at all in the builder pattern I
>>>>> assume I can reuse everything.
>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan
>> of
>>>>> the grouped.
>>>>>
>>>>> Yes, grouped() was more for demonstration and because groupBy() and
>>>> groupByKey() were taken! So i'd imagine the api would actually want to
>> be
>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>> depends
>>>> on maintaining backward compatibility.
>>>>
>>>>
>>>>> Unfortunately, the below approach would require atleast 2 (probably 3)
>>>>> overloads (one for returning a KTable and one for returning a KTable
>> with
>>>>> Windowed Key, probably would want to split windowed and sessionwindowed
>>>> for
>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>> Obviously not exhaustive but enough for you to get the picture. Count,
>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>> builder:
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Windowed Count
>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>
>>>>
>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>>>>>
>>>>> // Session Count
>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>
>>>>
>> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>
>>>>>
>>>> Above and below, i think i'd prefer it to be:
>>>> groupedStream.count(/** non windowed count**/)
>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>
>>>>
>>>>
>>>>
>>>>> // Reduce
>>>>> Reducer<Long> reducer;
>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Aggregate Windowed with Custom Store
>>>>> Initializer<String> initializer;
>>>>> Aggregator<String, Long, String> aggregator;
>>>>> KTable<Windowed<String>, String> aggregate =
>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>
>>>>
>> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>>>>>
>>>>> // Cogroup SessionWindowed
>>>>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>>>>>         .cogroup(groupedStream2, aggregator2)
>>>>>         .aggregate(initializer, aggregator,
>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>
>>>>>
>>>>>
>>>>> public class Count {
>>>>>
>>>>>     public static class Windowed extends Count {
>>>>>         private Windows windows;
>>>>>     }
>>>>>     public static class SessionWindowed extends Count {
>>>>>         private SessionWindows sessionWindows;
>>>>>     }
>>>>>
>>>>>     public static Count count();
>>>>>     public static Windowed windowed(Windows windows);
>>>>>     public static SessionWindowed sessionWindowed(SessionWindows
>>>>> sessionWindows);
>>>>>
>>>>>     // All withXXX(...) methods.
>>>>> }
>>>>>
>>>>> public class KGroupedStream {
>>>>>     public KTable<K, Long> count(Count count);
>>>>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>> count);
>>>>> …
>>>>> }
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> From: Guozhang Wang
>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>> To: dev@kafka.apache.org
>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>
>>>>> I played the current proposal a bit with
>> https://github.com/dguy/kafka/
>>>>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment
>>> ,
>>>>> and here are my observations:
>>>>>
>>>>> 1. Personally I prefer
>>>>>
>>>>>     "stream.group(mapper) / stream.groupByKey()"
>>>>>
>>>>> than
>>>>>
>>>>>     "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>
>>>>> Since 1) withKeyMapper is not enforced programmatically though it is
>> not
>>>>> "really" optional like others, 2) syntax-wise it reads more natural.
>>>>>
>>>>> I think it is okay to add the APIs in (
>>>>>
>>>>>
>>>>
>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>> )
>>>>> in KGroupedStream.
>>>>>
>>>>>
>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass
>> in
>>>>> the most-inner state store supplier (e.g. then one whose get() return
>>>>> RocksDBStore), or it is supposed to return the most-outer supplier with
>>>>> logging / metrics / etc? I think it would be more useful to only
>> require
>>>>> users pass in the inner state store supplier while specifying caching /
>>>>> logging through other APIs.
>>>>>
>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
>>>> are
>>>>> allowing users to call other APIs like "withQueryableName" multiple
>> time,
>>>>> but only call "withStateStoreSupplier" only once in the end. Why is
>> that?
>>>>>
>>>>>
>>>>> 3. The current DSL seems to be only for aggregations, what about joins?
>>>>>
>>>>>
>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>> StateStoreSupplier it will still be user code specifying the topology
>> so
>>>> I
>>>>> do not see there is a big difference.
>>>>>
>>>>>
>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
>>>>> windowed state store supplier to enforce typing?
>>>>>
>>>>>
>>>>> Below are minor ones:
>>>>>
>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>
>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> I see your point about "when to add the processor to the topology".
>>>> That
>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>> topology...
>>>>>>
>>>>>> I don't see any problem with having all the withXX() in KTable
>>>> interface
>>>>>> -- but this might be subjective.
>>>>>>
>>>>>>
>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>> after the withXX() -- all the calls to withXX() set optional
>> parameters
>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>>>> be quite confusion for developers.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>> I also think that mixing optional parameters with configs is a bad
>>>>> idea.
>>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
>>>> to
>>>>>>>> find some time to come up with something.
>>>>>>>>
>>>>>>>>
>>>>>>> Yes, i don't like the mix of config either. But the only real config
>>>>> here
>>>>>>> is the logging config - which we don't really need as it can already
>>>> be
>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>
>>>>>>>
>>>>>>>> What I don't like in the current proposal is the
>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>> .groupBy(...)
>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>>>>>>>> some better names).
>>>>>>>>
>>>>>>>>
>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> The proposed pattern "chains" grouping and aggregation too close
>>>>>>>> together. I would rather separate both more than less, ie, do into
>>>> the
>>>>>>>> opposite direction.
>>>>>>>>
>>>>>>>> I am also wondering, if we could so something more "fluent". The
>>>>> initial
>>>>>>>> proposal was like:
>>>>>>>>
>>>>>>>>>> groupedStream.count()
>>>>>>>>>>    .withStoreName("name")
>>>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>>>    .table()
>>>>>>>>
>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>
>>>>>>>
>>>>>>> I agree, but then all of the withXXX methods need to be on KTable
>>>> which
>>>>>> is
>>>>>>> worse in my opinion. You also need something that is going to "build"
>>>>> the
>>>>>>> internal processors and add them to the topology.
>>>>>>>
>>>>>>>
>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>> optional
>>>>>>>> parameter for count() have to specified on the .grouped() call --
>>>> this
>>>>>>>> does not seems to be the best way either.
>>>>>>>>
>>>>>>>>
>>>>>>> I actually prefer this method as you are building a grouped stream
>>>> that
>>>>>> you
>>>>>>> will aggregate. So
>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>> etc
>>>>>>> seems natural to me.
>>>>>>>
>>>>>>>
>>>>>>>> I did not think this through in detail, but can't we just do the
>>>>> initial
>>>>>>>> proposal with the .table() ?
>>>>>>>>
>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>
>>>>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
>>>>> are
>>>>>>>> just added to the KTable interface. Or do I miss anything why this
>>>>> wont'
>>>>>>>> work or any obvious disadvantage?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>> See above.
>>>>>>>
>>>>>>>
>>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>> fluent
>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
>>>> the
>>>>>> DSL,
>>>>>>>>> but i think that enabling caching and enabling logging are things
>>>>> that
>>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
>>>> (even
>>>>>>>>> though it is below) as this is actually config and we already have
>>>> a
>>>>>> way
>>>>>>>> of
>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
>>>>> is
>>>>>> a
>>>>>>>>> bit of a tedious process for someone that just wants to use the
>>>>> default
>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>
>>>>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
>>>>> you
>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>> logged
>>>>>> etc,
>>>>>>>>> you would currently need to implement that yourself. Ideally we can
>>>>>>>> provide
>>>>>>>>> a way where we will wrap the custom store with logging, metrics,
>>>>> etc. I
>>>>>>>>> need to think about where this fits, it is probably more
>>>> appropriate
>>>>> on
>>>>>>>> the
>>>>>>>>> Stores API.
>>>>>>>>>
>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>> // count with mapped key
>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>         .withKeyMapper(keyMapper)
>>>>>>>>>         .withKeySerde(Serdes.Long())
>>>>>>>>>         .withValueSerde(Serdes.String())
>>>>>>>>>         .withQueryableName("my-store")
>>>>>>>>>         .count();
>>>>>>>>>
>>>>>>>>> // windowed count
>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>> stream.grouped()
>>>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>         .count();
>>>>>>>>>
>>>>>>>>> // windowed reduce
>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>> stream.grouped()
>>>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>         .reduce(windowedReducer);
>>>>>>>>>
>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>
>>>>>>>>> // aggregate
>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>         .withQueryableName("my-aggregate-store")
>>>>>>>>>         .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>
>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>> stateStoreSupplier
>>>>>>>> = null;
>>>>>>>>>
>>>>>>>>> // aggregate with custom store
>>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>         .aggregate(aggregator, init);
>>>>>>>>>
>>>>>>>>> // disable caching
>>>>>>>>> stream.grouped()
>>>>>>>>>         .withQueryableName("name")
>>>>>>>>>         .withCachingEnabled(false)
>>>>>>>>>         .count();
>>>>>>>>>
>>>>>>>>> // disable logging
>>>>>>>>> stream.grouped()
>>>>>>>>>         .withQueryableName("q")
>>>>>>>>>         .withLoggingEnabled(false)
>>>>>>>>>         .count();
>>>>>>>>>
>>>>>>>>> // override log config
>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>> stream.grouped()
>>>>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
>>>>> "10"))
>>>>>>>>>         .reduce(reducer);
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> If anyone wants to play around with this you can find the code
>>>> here:
>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>
>>>>>>>>> Note: It won't actually work as most of the methods just return
>>>> null.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
>>>> are
>>>>>>>> better
>>>>>>>>>> than overload abuse.
>>>>>>>>>>
>>>>>>>>>> The fluent API approach reads better, no mention of builder or
>>>> build
>>>>>>>>>> anywhere. The main downside is that the method signatures are a
>>>>> little
>>>>>>>> less
>>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
>>>>> knows
>>>>>>>> what
>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>> (`table()`
>>>>>>>> in
>>>>>>>>>> this case) that gives you what you actually care about (`KTable`
>>>> in
>>>>>> this
>>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>>>> comparison.
>>>>>>>>>>
>>>>>>>>>> The builder approach avoids the issues mentioned above, but it
>>>>> doesn't
>>>>>>>> read
>>>>>>>>>> as well.
>>>>>>>>>>
>>>>>>>>>> Ismael
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>>>
>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> I'd like to get a discussion going around some of the API choices
>>>>>> we've
>>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>>>> operations
>>>>>>>>>>> (though this could expand).
>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
>>>> i.e,
>>>>>>>> there
>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>> noisy
>>>>>> and
>>>>>>>> i
>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>> params.
>>>>> In
>>>>>>>>>>> particular we've had some requests to be able to turn caching
>>>> off,
>>>>> or
>>>>>>>>>>> change log configs,  on a per operator basis (note this can be
>>>> done
>>>>>> now
>>>>>>>>>> if
>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>> cumbersome).
>>>>>>>>>>>
>>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>>>>> overloads
>>>>>>>>>>> so that it flows, is simple to use and understand, and is easily
>>>>>>>> extended
>>>>>>>>>>> in the future?
>>>>>>>>>>>
>>>>>>>>>>> One option would be to use a fluent API approach for providing
>>>> the
>>>>>>>>>> optional
>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>
>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>    .withStoreName("name")
>>>>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>>>>    .table()
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Another option would be to provide a Builder to the count method,
>>>>> so
>>>>>> it
>>>>>>>>>>> would look something like this:
>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>>>>>
>>>>>>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>>>>>> about!
>>>>>>>>>>>
>>>>>>>>>>> The above has focussed on state store related overloads, but the
>>>>> same
>>>>>>>>>> ideas
>>>>>>>>>>> could  be applied to joins etc, where we presently have many join
>>>>>>>> methods
>>>>>>>>>>> and many overloads.
>>>>>>>>>>>
>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>>
>>>>
>>>>
>>>
>>
>>
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Jan Filipiak <Ja...@trivago.com>.
Hi Damian,

I do see your point of something needs to change. But I fully agree with 
Gouzhang when he says.
---

But since this is a incompatibility change, and we are going to remove the
compatibility annotations soon it means we only have one chance and we
really have to make it right.
----

I fear all suggestions do not go far enough to become something that will carry on for very much longer.
I am currently working on KAFKA-3705 and try to find the most easy way for the user to give me all the required functionality. The easiest interface I could come up so far can be looked at here.

https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L622

And its already horribly complicated. I am currently unable to find the right abstraction level to have everything falling into place naturally. To be honest I already think introducing

https://github.com/Kaiserchen/kafka/blob/3da2b8f787a5d30dee2de71cf0f125ab3e57d89b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L493

was unideal and makes everything a mess. The JoinType:Whatever is also not really flexible. 2 things come to my mind:

1. I don't think we should rule out config based decisions say configs like
	streams.$applicationID.joins.$joinname.conf = value
This can allow for tremendous changes without single API change and IMO it was not considered enough yet.

2. Push logic from the DSL to the Callback classes. A ValueJoiner for example can be used to implement different join types as the user wishes.

As Gouzhang said: stopping to break users is very important. especially with this changes + All the plans I sadly only have in my head but hopefully the first link can give a glimpse.

Thanks for preparing the examples made it way clearer to me what exactly we are talking about. I would argue to go a bit slower and more carefull on this one. At some point we need to get it right. Peeking over to the hadoop guys with their hughe userbase. Config files really work well for them.

Best Jan





On 30.06.2017 09:31, Damian Guy wrote:
> Thanks Matthias
>
> On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io> wrote:
>
>> I am just catching up on this thread, so sorry for the long email in
>> advance... Also, it's to some extend a dump of thoughts and not always a
>> clear proposal. Still need to think about this in more detail. But maybe
>> it helps other to get new ideas :)
>>
>>
>>>> However, I don't understand your argument about putting aggregate()
>>>> after the withXX() -- all the calls to withXX() set optional parameters
>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>> be quite confusion for developers.
>>>>
>>>>
>>> I see what you are saying, but the grouped stream is effectively a no-op
>>> until you call one of the aggregate/count/reduce etc functions. So the
>>> optional params are ones that are applicable to any of the operations you
>>> can perform on this grouped stream. Then the final
>>> count()/reduce()/aggregate() call has any of the params that are
>>> required/specific to that function.
>>>
>> I understand your argument, but you don't share the conclusion. If we
>> need a "final/terminal" call, the better way might be
>>
>> .groupBy().count().withXX().build()
>>
>> (with a better name for build() though)
>>
>>
> The point is that all the other calls, i.e,withBlah, windowed, etc apply
> too all the aggregate functions. The terminal call being the actual type of
> aggregation you want to do. I personally find this more natural than
> groupBy().count().withBlah().build()
>
>
>>> groupedStream.count(/** non windowed count**/)
>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>
>> I like this. However, I don't see a reason to have windowed() and
>> sessionWindowed(). We should have one top-level `Windows` interface that
>> both `TimeWindows` and `SessionWindows` implement and just have a single
>> windowed() method that accepts all `Windows`. (I did not like the
>> separation of `SessionWindows` in the first place, and this seems to be
>> an opportunity to clean this up. It was hard to change when we
>> introduced session windows)
>>
> Yes - true we should look into that.
>
>
>> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
>> might also want to use windowBy() (instead of windowed()). Not sure how
>> important this is, but it seems to be inconsistent otherwise.
>>
>>
> Makes sense
>
>
>> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
>> defining an inner/left/outer join is not an optional argument but a
>> first class concept and should have a proper representation in the API
>> (like the current methods join(), leftJoin, outerJoin()).
>>
>>
> Yep, i did originally have it as a required param and maybe that is what we
> go with. It could have a default, but maybe that is confusing.
>
>
>
>> About the two join API proposals, the second one has too much boiler
>> plate code for my taste. Also, the actual join() operator has only one
>> argument what is weird to me, as in my thinking process, the main
>> operator call, should have one parameter per mandatory argument but your
>> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
>> This is far from intuitive IMHO.
>>
>>
> This is the builder pattern, you only need one param as the builder has
> captured all of the required and optional arguments.
>
>
>> The first join proposal also seems to align better with the pattern
>> suggested for aggregations and having the same pattern for all operators
>> is important (as you stated already).
>>
>>
> This is why i offered two alternatives as i started out with. 1 is the
> builder pattern, the other is the more fluent pattern.
>
>
>>
>> Coming back to the config vs optional parameter. What about having a
>> method withConfig[s](...) that allow to put in the configuration?
>>
>>
> Sure, it is currently called withLogConfig() as that is the only thing that
> is really config.
>
>
>> This also raises the question if until() is a windows property?
>> Actually, until() seems to be a configuration parameter and thus, should
>> not not have it's own method.
>>
>>
> Hmmm, i don't agree. Until is a property of the window. It is going to be
> potentially different for every window operation you do in a streams app.
>
>
>>
>> Browsing throw your example DSL branch, I also saw this one:
>>
>>> final KTable<Windowed<String>, Long> windowed>
>>   groupedStream.counting()
>>>                   .windowed(TimeWindows.of(10L).until(10))
>>>                   .table();
>> This is an interesting idea, and it remind my on some feedback about "I
>> wanted to count a stream, but there was no count() method -- I first
>> needed to figure out, that I need to group the stream first to be able
>> to count it. It does make sense in hindsight but was not obvious in the
>> beginning". Thus, carrying out this thought, we could also do the
>> following:
>>
>> stream.count().groupedBy().windowedBy().table();
>>
>> -> Note, I use "grouped" and "windowed" instead of imperative here, as
>> it comes after the count()
>>
>> This would be more consistent than your proposal (that has grouping
>> before but windowing after count()). It might even allow us to enrich
>> the API with a some syntactic sugar like `stream.count().table()` to get
>> the overall count of all records (this would obviously not scale, but we
>> could support it -- if not now, maybe later).
>>
>>
> I guess i'd prefer
> stream.groupBy().windowBy().count()
> stream.groupBy().windowBy().reduce()
> stream.groupBy().count()
>
> As i said above, everything that happens before the final aggregate call
> can be applied to any of them. So it makes sense to me to do those things
> ahead of the final aggregate call.
>
>
>> Last about builder pattern. I am convinced that we need some "terminal"
>> operator/method that tells us when to add the processor to the topology.
>> But I don't see the need for a plain builder pattern that feels alien to
>> me (see my argument about the second join proposal). Using .stream() /
>> .table() as use in many examples might work. But maybe a more generic
>> name that we can use in all places like build() or apply() might also be
>> an option.
>>
>>
> Sure, a generic name might be ok.
>
>
>
>
>> -Matthias
>>
>>
>>
>> On 6/29/17 7:37 AM, Damian Guy wrote:
>>> Thanks Kyle.
>>>
>>> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
>>> wrote:
>>>
>>>> Hi Damian,
>>>>
>>>>>>>> When trying to program in the fluent API that has been discussed
>> most
>>>> it
>>>>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
>> to
>>>>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>>> I'd anticipate that once you have a KGroupedStream you can re-use it
>> as
>>>> you
>>>>>> can today.
>>>> You said it yourself in another post that the grouped stream is
>>>> effectively a no-op until a count, reduce, or aggregate. The way I see
>> it
>>>> you wouldn’t be able to reuse anything except KStreams and KTables,
>> because
>>>> most of this fluent api would continue returning this (this being the
>>>> builder object currently being manipulated).
>>> So, if you ever store a reference to anything but KStreams and KTables
>> and
>>>> you use it in two different ways then its possible you make conflicting
>>>> withXXX() calls on the same builder.
>>>>
>>>>
>>> No necessarily true. It could return a new instance of the builder, i.e.,
>>> the builders being immutable. So if you held a reference to the builder
>> it
>>> would always be the same as it was when it was created.
>>>
>>>
>>>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
>>>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>>>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>>>
>>>> I’ll admit that this shouldn’t happen but some user is going to do it
>>>> eventually…
>>>> Depending on implementation uses of groupedStreamWithDefaultSerdes would
>>>> most likely be equivalent to the version withDeclaredSerdes. One work
>>>> around would be to always make copies of the config objects you are
>>>> building, but this approach has its own problem because now we have to
>>>> identify which configs are equivalent so we don’t create repeated
>>>> processors.
>>>>
>>>> The point of this long winded example is that we always have to be
>>>> thinking about all of the possible ways it could be misused by a user
>>>> (causing them to see hard to diagnose problems).
>>>>
>>> Exactly! That is the point of the discussion really.
>>>
>>>
>>>> In my attempt at a couple methods with builders I feel that I could
>>>> confidently say the user couldn’t really mess it up.
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>> The kGroupedStream is reusable and if they attempted to reuse the Count
>>>> for some reason it would throw an error message saying that a store
>> named
>>>> “my-store” already exists.
>>>>
>>>>
>>> Yes i agree and i think using builders is my preferred pattern.
>>>
>>> Cheers,
>>> Damian
>>>
>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> From: Damian Guy
>>>> Sent: Thursday, June 29, 2017 3:59 AM
>>>> To: dev@kafka.apache.org
>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>
>>>> Hi Kyle,
>>>>
>>>> Thanks for your input. Really appreciated.
>>>>
>>>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
>>>> wrote:
>>>>
>>>>> I like more of a builder pattern even though others have voiced against
>>>>> it. The reason I like it is because it makes it clear to the user that
>> a
>>>>> call to KGroupedStream#count will return a KTable not some intermediate
>>>>> class that I need to undetstand.
>>>>>
>>>> Yes, that makes sense.
>>>>
>>>>
>>>>> When trying to program in the fluent API that has been discussed most
>> it
>>>>> feels difficult to know when you will actually get an object you can
>>>> reuse.
>>>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>>>> reuse it or does this approach expect you to call grouped each time?
>>>>
>>>> I'd anticipate that once you have a KGroupedStream you can re-use it as
>> you
>>>> can today.
>>>>
>>>>
>>>>> This question doesn’t pop into my head at all in the builder pattern I
>>>>> assume I can reuse everything.
>>>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan
>> of
>>>>> the grouped.
>>>>>
>>>>> Yes, grouped() was more for demonstration and because groupBy() and
>>>> groupByKey() were taken! So i'd imagine the api would actually want to
>> be
>>>> groupByKey(/** no required args***/).withOptionalArg() and
>>>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
>> depends
>>>> on maintaining backward compatibility.
>>>>
>>>>
>>>>> Unfortunately, the below approach would require atleast 2 (probably 3)
>>>>> overloads (one for returning a KTable and one for returning a KTable
>> with
>>>>> Windowed Key, probably would want to split windowed and sessionwindowed
>>>> for
>>>>> ease of implementation) of each count, reduce, and aggregate.
>>>>> Obviously not exhaustive but enough for you to get the picture. Count,
>>>>> Reduce, and Aggregate supply 3 static methods to initialize the
>> builder:
>>>>> // Count
>>>>> KTable<String, Long> count =
>>>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Windowed Count
>>>>> KTable<Windowed<String>, Long> windowedCount =
>>>>>
>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>>>>> // Session Count
>>>>> KTable<Windowed<String>, Long> sessionCount =
>>>>>
>> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>>>
>>>> Above and below, i think i'd prefer it to be:
>>>> groupedStream.count(/** non windowed count**/)
>>>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>>>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>>>
>>>>
>>>>
>>>>
>>>>> // Reduce
>>>>> Reducer<Long> reducer;
>>>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>>>
>>>>> // Aggregate Windowed with Custom Store
>>>>> Initializer<String> initializer;
>>>>> Aggregator<String, Long, String> aggregator;
>>>>> KTable<Windowed<String>, String> aggregate =
>>>>> groupedStream.aggregate(initializer, aggregator,
>>>>>
>> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>>>>> // Cogroup SessionWindowed
>>>>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>>>>>          .cogroup(groupedStream2, aggregator2)
>>>>>          .aggregate(initializer, aggregator,
>>>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>>>> sessionMerger).withQueryableStoreName("my-store"));
>>>>>
>>>>>
>>>>>
>>>>> public class Count {
>>>>>
>>>>>      public static class Windowed extends Count {
>>>>>          private Windows windows;
>>>>>      }
>>>>>      public static class SessionWindowed extends Count {
>>>>>          private SessionWindows sessionWindows;
>>>>>      }
>>>>>
>>>>>      public static Count count();
>>>>>      public static Windowed windowed(Windows windows);
>>>>>      public static SessionWindowed sessionWindowed(SessionWindows
>>>>> sessionWindows);
>>>>>
>>>>>      // All withXXX(...) methods.
>>>>> }
>>>>>
>>>>> public class KGroupedStream {
>>>>>      public KTable<K, Long> count(Count count);
>>>>>      public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>>>      public KTable<Windowed<K>, Long> count(Count.SessionWindowed
>> count);
>>>>> …
>>>>> }
>>>>>
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> From: Guozhang Wang
>>>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>>>> To: dev@kafka.apache.org
>>>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>>>
>>>>> I played the current proposal a bit with
>> https://github.com/dguy/kafka/
>>>>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment
>>> ,
>>>>> and here are my observations:
>>>>>
>>>>> 1. Personally I prefer
>>>>>
>>>>>      "stream.group(mapper) / stream.groupByKey()"
>>>>>
>>>>> than
>>>>>
>>>>>      "stream.group().withKeyMapper(mapper) / stream.group()"
>>>>>
>>>>> Since 1) withKeyMapper is not enforced programmatically though it is
>> not
>>>>> "really" optional like others, 2) syntax-wise it reads more natural.
>>>>>
>>>>> I think it is okay to add the APIs in (
>>>>>
>>>>>
>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>>>> )
>>>>> in KGroupedStream.
>>>>>
>>>>>
>>>>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass
>> in
>>>>> the most-inner state store supplier (e.g. then one whose get() return
>>>>> RocksDBStore), or it is supposed to return the most-outer supplier with
>>>>> logging / metrics / etc? I think it would be more useful to only
>> require
>>>>> users pass in the inner state store supplier while specifying caching /
>>>>> logging through other APIs.
>>>>>
>>>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
>>>> are
>>>>> allowing users to call other APIs like "withQueryableName" multiple
>> time,
>>>>> but only call "withStateStoreSupplier" only once in the end. Why is
>> that?
>>>>>
>>>>> 3. The current DSL seems to be only for aggregations, what about joins?
>>>>>
>>>>>
>>>>> 4. I think it is okay to keep the "withLogConfig": for the
>>>>> StateStoreSupplier it will still be user code specifying the topology
>> so
>>>> I
>>>>> do not see there is a big difference.
>>>>>
>>>>>
>>>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
>>>>> windowed state store supplier to enforce typing?
>>>>>
>>>>>
>>>>> Below are minor ones:
>>>>>
>>>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>>>
>>>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>>>
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
>> matthias@confluent.io>
>>>>> wrote:
>>>>>
>>>>>> I see your point about "when to add the processor to the topology".
>>>> That
>>>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>>>> topology...
>>>>>> I don't see any problem with having all the withXX() in KTable
>>>> interface
>>>>>> -- but this might be subjective.
>>>>>>
>>>>>>
>>>>>> However, I don't understand your argument about putting aggregate()
>>>>>> after the withXX() -- all the calls to withXX() set optional
>> parameters
>>>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>>>> be quite confusion for developers.
>>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>>>> I also think that mixing optional parameters with configs is a bad
>>>>> idea.
>>>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
>>>> to
>>>>>>>> find some time to come up with something.
>>>>>>>>
>>>>>>>>
>>>>>>> Yes, i don't like the mix of config either. But the only real config
>>>>> here
>>>>>>> is the logging config - which we don't really need as it can already
>>>> be
>>>>>>> done via a custom StateStoreSupplier.
>>>>>>>
>>>>>>>
>>>>>>>> What I don't like in the current proposal is the
>>>>>>>> .grouped().withKeyMapper() -- the current solution with
>>>> .groupBy(...)
>>>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>>>>>>>> some better names).
>>>>>>>>
>>>>>>>>
>>>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>> The proposed pattern "chains" grouping and aggregation too close
>>>>>>>> together. I would rather separate both more than less, ie, do into
>>>> the
>>>>>>>> opposite direction.
>>>>>>>>
>>>>>>>> I am also wondering, if we could so something more "fluent". The
>>>>> initial
>>>>>>>> proposal was like:
>>>>>>>>
>>>>>>>>>> groupedStream.count()
>>>>>>>>>>     .withStoreName("name")
>>>>>>>>>>     .withCachingEnabled(false)
>>>>>>>>>>     .withLoggingEnabled(config)
>>>>>>>>>>     .table()
>>>>>>>> The .table() statement in the end was kinda alien.
>>>>>>>>
>>>>>>> I agree, but then all of the withXXX methods need to be on KTable
>>>> which
>>>>>> is
>>>>>>> worse in my opinion. You also need something that is going to "build"
>>>>> the
>>>>>>> internal processors and add them to the topology.
>>>>>>>
>>>>>>>
>>>>>>>> The current proposal put the count() into the end -- ie, the
>>>> optional
>>>>>>>> parameter for count() have to specified on the .grouped() call --
>>>> this
>>>>>>>> does not seems to be the best way either.
>>>>>>>>
>>>>>>>>
>>>>>>> I actually prefer this method as you are building a grouped stream
>>>> that
>>>>>> you
>>>>>>> will aggregate. So
>>>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>>>> etc
>>>>>>> seems natural to me.
>>>>>>>
>>>>>>>
>>>>>>>> I did not think this through in detail, but can't we just do the
>>>>> initial
>>>>>>>> proposal with the .table() ?
>>>>>>>>
>>>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>>>
>>>>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
>>>>> are
>>>>>>>> just added to the KTable interface. Or do I miss anything why this
>>>>> wont'
>>>>>>>> work or any obvious disadvantage?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>> See above.
>>>>>>>
>>>>>>>
>>>>>>>> -Matthias
>>>>>>>>
>>>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>>>> fluent
>>>>>>>>> approach, but i think it is slightly nicer.
>>>>>>>>> I agree with some of what Eno said about mixing configy stuff in
>>>> the
>>>>>> DSL,
>>>>>>>>> but i think that enabling caching and enabling logging are things
>>>>> that
>>>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
>>>> (even
>>>>>>>>> though it is below) as this is actually config and we already have
>>>> a
>>>>>> way
>>>>>>>> of
>>>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
>>>>> is
>>>>>> a
>>>>>>>>> bit of a tedious process for someone that just wants to use the
>>>>> default
>>>>>>>>> storage engine, but not have caching enabled.
>>>>>>>>>
>>>>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
>>>>> you
>>>>>>>>> want to plug in a custom storage engine and you want it to be
>>>> logged
>>>>>> etc,
>>>>>>>>> you would currently need to implement that yourself. Ideally we can
>>>>>>>> provide
>>>>>>>>> a way where we will wrap the custom store with logging, metrics,
>>>>> etc. I
>>>>>>>>> need to think about where this fits, it is probably more
>>>> appropriate
>>>>> on
>>>>>>>> the
>>>>>>>>> Stores API.
>>>>>>>>>
>>>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>>>> // count with mapped key
>>>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>>>          .withKeyMapper(keyMapper)
>>>>>>>>>          .withKeySerde(Serdes.Long())
>>>>>>>>>          .withValueSerde(Serdes.String())
>>>>>>>>>          .withQueryableName("my-store")
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // windowed count
>>>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("my-window-store")
>>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // windowed reduce
>>>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("my-window-store")
>>>>>>>>>          .windowed(TimeWindows.of(10L).until(10))
>>>>>>>>>          .reduce(windowedReducer);
>>>>>>>>>
>>>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>>>> final Initializer<Long> init = null;
>>>>>>>>>
>>>>>>>>> // aggregate
>>>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>>>          .withQueryableName("my-aggregate-store")
>>>>>>>>>          .aggregate(aggregator, init, Serdes.Long());
>>>>>>>>>
>>>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>>>> stateStoreSupplier
>>>>>>>> = null;
>>>>>>>>> // aggregate with custom store
>>>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>>>>>>>          .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>>>          .aggregate(aggregator, init);
>>>>>>>>>
>>>>>>>>> // disable caching
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("name")
>>>>>>>>>          .withCachingEnabled(false)
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // disable logging
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withQueryableName("q")
>>>>>>>>>          .withLoggingEnabled(false)
>>>>>>>>>          .count();
>>>>>>>>>
>>>>>>>>> // override log config
>>>>>>>>> final Reducer<String> reducer = null;
>>>>>>>>> stream.grouped()
>>>>>>>>>          .withLogConfig(Collections.singletonMap("segment.size",
>>>>> "10"))
>>>>>>>>>          .reduce(reducer);
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> If anyone wants to play around with this you can find the code
>>>> here:
>>>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>>>
>>>>>>>>> Note: It won't actually work as most of the methods just return
>>>> null.
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>>>> wrote:
>>>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
>>>> are
>>>>>>>> better
>>>>>>>>>> than overload abuse.
>>>>>>>>>>
>>>>>>>>>> The fluent API approach reads better, no mention of builder or
>>>> build
>>>>>>>>>> anywhere. The main downside is that the method signatures are a
>>>>> little
>>>>>>>> less
>>>>>>>>>> clear. By reading the method signature, one doesn't necessarily
>>>>> knows
>>>>>>>> what
>>>>>>>>>> it returns. Also, one needs to figure out the special method
>>>>>> (`table()`
>>>>>>>> in
>>>>>>>>>> this case) that gives you what you actually care about (`KTable`
>>>> in
>>>>>> this
>>>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>>>> comparison.
>>>>>>>>>> The builder approach avoids the issues mentioned above, but it
>>>>> doesn't
>>>>>>>> read
>>>>>>>>>> as well.
>>>>>>>>>>
>>>>>>>>>> Ismael
>>>>>>>>>>
>>>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>>>>>> wrote:
>>>>>>>>>>> Hi,
>>>>>>>>>>>
>>>>>>>>>>> I'd like to get a discussion going around some of the API choices
>>>>>> we've
>>>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>>>> operations
>>>>>>>>>>> (though this could expand).
>>>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
>>>> i.e,
>>>>>>>> there
>>>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>>>> noisy
>>>>>> and
>>>>>>>> i
>>>>>>>>>>> feel it is only going to get worse as we add more optional
>>>> params.
>>>>> In
>>>>>>>>>>> particular we've had some requests to be able to turn caching
>>>> off,
>>>>> or
>>>>>>>>>>> change log configs,  on a per operator basis (note this can be
>>>> done
>>>>>> now
>>>>>>>>>> if
>>>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>>>> cumbersome).
>>>>>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>>>>> overloads
>>>>>>>>>>> so that it flows, is simple to use and understand, and is easily
>>>>>>>> extended
>>>>>>>>>>> in the future?
>>>>>>>>>>>
>>>>>>>>>>> One option would be to use a fluent API approach for providing
>>>> the
>>>>>>>>>> optional
>>>>>>>>>>> params, so something like this:
>>>>>>>>>>>
>>>>>>>>>>> groupedStream.count()
>>>>>>>>>>>     .withStoreName("name")
>>>>>>>>>>>     .withCachingEnabled(false)
>>>>>>>>>>>     .withLoggingEnabled(config)
>>>>>>>>>>>     .table()
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>>> Another option would be to provide a Builder to the count method,
>>>>> so
>>>>>> it
>>>>>>>>>>> would look something like this:
>>>>>>>>>>> groupedStream.count(new
>>>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>>>>>
>>>>>>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>>>>>> about!
>>>>>>>>>>> The above has focussed on state store related overloads, but the
>>>>> same
>>>>>>>>>> ideas
>>>>>>>>>>> could  be applied to joins etc, where we presently have many join
>>>>>>>> methods
>>>>>>>>>>> and many overloads.
>>>>>>>>>>>
>>>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Damian
>>>>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>
>>>>> --
>>>>> -- Guozhang
>>>>>
>>>>>
>>>>
>>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks Matthias

On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io> wrote:

> I am just catching up on this thread, so sorry for the long email in
> advance... Also, it's to some extend a dump of thoughts and not always a
> clear proposal. Still need to think about this in more detail. But maybe
> it helps other to get new ideas :)
>
>
> >> However, I don't understand your argument about putting aggregate()
> >> after the withXX() -- all the calls to withXX() set optional parameters
> >> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >> be quite confusion for developers.
> >>
> >>
> > I see what you are saying, but the grouped stream is effectively a no-op
> > until you call one of the aggregate/count/reduce etc functions. So the
> > optional params are ones that are applicable to any of the operations you
> > can perform on this grouped stream. Then the final
> > count()/reduce()/aggregate() call has any of the params that are
> > required/specific to that function.
> >
>
> I understand your argument, but you don't share the conclusion. If we
> need a "final/terminal" call, the better way might be
>
> .groupBy().count().withXX().build()
>
> (with a better name for build() though)
>
>
The point is that all the other calls, i.e,withBlah, windowed, etc apply
too all the aggregate functions. The terminal call being the actual type of
aggregation you want to do. I personally find this more natural than
groupBy().count().withBlah().build()


> > groupedStream.count(/** non windowed count**/)
> > groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>
> I like this. However, I don't see a reason to have windowed() and
> sessionWindowed(). We should have one top-level `Windows` interface that
> both `TimeWindows` and `SessionWindows` implement and just have a single
> windowed() method that accepts all `Windows`. (I did not like the
> separation of `SessionWindows` in the first place, and this seems to be
> an opportunity to clean this up. It was hard to change when we
> introduced session windows)
>

Yes - true we should look into that.


>
> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> might also want to use windowBy() (instead of windowed()). Not sure how
> important this is, but it seems to be inconsistent otherwise.
>
>
Makes sense


>
> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
> defining an inner/left/outer join is not an optional argument but a
> first class concept and should have a proper representation in the API
> (like the current methods join(), leftJoin, outerJoin()).
>
>
Yep, i did originally have it as a required param and maybe that is what we
go with. It could have a default, but maybe that is confusing.



> About the two join API proposals, the second one has too much boiler
> plate code for my taste. Also, the actual join() operator has only one
> argument what is weird to me, as in my thinking process, the main
> operator call, should have one parameter per mandatory argument but your
> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
> This is far from intuitive IMHO.
>
>
This is the builder pattern, you only need one param as the builder has
captured all of the required and optional arguments.


> The first join proposal also seems to align better with the pattern
> suggested for aggregations and having the same pattern for all operators
> is important (as you stated already).
>
>
This is why i offered two alternatives as i started out with. 1 is the
builder pattern, the other is the more fluent pattern.


>
>
> Coming back to the config vs optional parameter. What about having a
> method withConfig[s](...) that allow to put in the configuration?
>
>
Sure, it is currently called withLogConfig() as that is the only thing that
is really config.


> This also raises the question if until() is a windows property?
> Actually, until() seems to be a configuration parameter and thus, should
> not not have it's own method.
>
>
Hmmm, i don't agree. Until is a property of the window. It is going to be
potentially different for every window operation you do in a streams app.


>
>
> Browsing throw your example DSL branch, I also saw this one:
>
> > final KTable<Windowed<String>, Long> windowed>
>  groupedStream.counting()
> >                  .windowed(TimeWindows.of(10L).until(10))
> >                  .table();
>
> This is an interesting idea, and it remind my on some feedback about "I
> wanted to count a stream, but there was no count() method -- I first
> needed to figure out, that I need to group the stream first to be able
> to count it. It does make sense in hindsight but was not obvious in the
> beginning". Thus, carrying out this thought, we could also do the
> following:
>
> stream.count().groupedBy().windowedBy().table();
>
> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> it comes after the count()
>
> This would be more consistent than your proposal (that has grouping
> before but windowing after count()). It might even allow us to enrich
> the API with a some syntactic sugar like `stream.count().table()` to get
> the overall count of all records (this would obviously not scale, but we
> could support it -- if not now, maybe later).
>
>
I guess i'd prefer
stream.groupBy().windowBy().count()
stream.groupBy().windowBy().reduce()
stream.groupBy().count()

As i said above, everything that happens before the final aggregate call
can be applied to any of them. So it makes sense to me to do those things
ahead of the final aggregate call.


> Last about builder pattern. I am convinced that we need some "terminal"
> operator/method that tells us when to add the processor to the topology.
> But I don't see the need for a plain builder pattern that feels alien to
> me (see my argument about the second join proposal). Using .stream() /
> .table() as use in many examples might work. But maybe a more generic
> name that we can use in all places like build() or apply() might also be
> an option.
>
>
Sure, a generic name might be ok.




>
> -Matthias
>
>
>
> On 6/29/17 7:37 AM, Damian Guy wrote:
> > Thanks Kyle.
> >
> > On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> > wrote:
> >
> >> Hi Damian,
> >>
> >>>>>> When trying to program in the fluent API that has been discussed
> most
> >> it
> >>>>>> feels difficult to know when you will actually get an object you can
> >> reuse.
> >>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> to
> >>>>>> reuse it or does this approach expect you to call grouped each time?
> >>
> >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> as
> >> you
> >>>> can today.
> >>
> >> You said it yourself in another post that the grouped stream is
> >> effectively a no-op until a count, reduce, or aggregate. The way I see
> it
> >> you wouldn’t be able to reuse anything except KStreams and KTables,
> because
> >> most of this fluent api would continue returning this (this being the
> >> builder object currently being manipulated).
> >
> > So, if you ever store a reference to anything but KStreams and KTables
> and
> >> you use it in two different ways then its possible you make conflicting
> >> withXXX() calls on the same builder.
> >>
> >>
> > No necessarily true. It could return a new instance of the builder, i.e.,
> > the builders being immutable. So if you held a reference to the builder
> it
> > would always be the same as it was when it was created.
> >
> >
> >> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> >> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> >> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >>
> >> I’ll admit that this shouldn’t happen but some user is going to do it
> >> eventually…
> >> Depending on implementation uses of groupedStreamWithDefaultSerdes would
> >> most likely be equivalent to the version withDeclaredSerdes. One work
> >> around would be to always make copies of the config objects you are
> >> building, but this approach has its own problem because now we have to
> >> identify which configs are equivalent so we don’t create repeated
> >> processors.
> >>
> >> The point of this long winded example is that we always have to be
> >> thinking about all of the possible ways it could be misused by a user
> >> (causing them to see hard to diagnose problems).
> >>
> >
> > Exactly! That is the point of the discussion really.
> >
> >
> >>
> >> In my attempt at a couple methods with builders I feel that I could
> >> confidently say the user couldn’t really mess it up.
> >>> // Count
> >>> KTable<String, Long> count =
> >>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >> The kGroupedStream is reusable and if they attempted to reuse the Count
> >> for some reason it would throw an error message saying that a store
> named
> >> “my-store” already exists.
> >>
> >>
> > Yes i agree and i think using builders is my preferred pattern.
> >
> > Cheers,
> > Damian
> >
> >
> >> Thanks,
> >> Kyle
> >>
> >> From: Damian Guy
> >> Sent: Thursday, June 29, 2017 3:59 AM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>
> >> Hi Kyle,
> >>
> >> Thanks for your input. Really appreciated.
> >>
> >> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
> >> wrote:
> >>
> >>> I like more of a builder pattern even though others have voiced against
> >>> it. The reason I like it is because it makes it clear to the user that
> a
> >>> call to KGroupedStream#count will return a KTable not some intermediate
> >>> class that I need to undetstand.
> >>>
> >>
> >> Yes, that makes sense.
> >>
> >>
> >>> When trying to program in the fluent API that has been discussed most
> it
> >>> feels difficult to know when you will actually get an object you can
> >> reuse.
> >>> What if I make one KGroupedStream that I want to reuse, is it legal to
> >>> reuse it or does this approach expect you to call grouped each time?
> >>
> >>
> >> I'd anticipate that once you have a KGroupedStream you can re-use it as
> you
> >> can today.
> >>
> >>
> >>> This question doesn’t pop into my head at all in the builder pattern I
> >>> assume I can reuse everything.
> >>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan
> of
> >>> the grouped.
> >>>
> >>> Yes, grouped() was more for demonstration and because groupBy() and
> >> groupByKey() were taken! So i'd imagine the api would actually want to
> be
> >> groupByKey(/** no required args***/).withOptionalArg() and
> >> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> depends
> >> on maintaining backward compatibility.
> >>
> >>
> >>> Unfortunately, the below approach would require atleast 2 (probably 3)
> >>> overloads (one for returning a KTable and one for returning a KTable
> with
> >>> Windowed Key, probably would want to split windowed and sessionwindowed
> >> for
> >>> ease of implementation) of each count, reduce, and aggregate.
> >>> Obviously not exhaustive but enough for you to get the picture. Count,
> >>> Reduce, and Aggregate supply 3 static methods to initialize the
> builder:
> >>> // Count
> >>> KTable<String, Long> count =
> >>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>
> >>> // Windowed Count
> >>> KTable<Windowed<String>, Long> windowedCount =
> >>>
> >>
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >>>
> >>> // Session Count
> >>> KTable<Windowed<String>, Long> sessionCount =
> >>>
> >>
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >>>
> >>>
> >> Above and below, i think i'd prefer it to be:
> >> groupedStream.count(/** non windowed count**/)
> >> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>
> >>
> >>
> >>
> >>> // Reduce
> >>> Reducer<Long> reducer;
> >>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> >>> Reduce.reduce().withQueryableStoreName("my-store"));
> >>>
> >>> // Aggregate Windowed with Custom Store
> >>> Initializer<String> initializer;
> >>> Aggregator<String, Long, String> aggregator;
> >>> KTable<Windowed<String>, String> aggregate =
> >>> groupedStream.aggregate(initializer, aggregator,
> >>>
> >>
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >>>
> >>> // Cogroup SessionWindowed
> >>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
> >>>         .cogroup(groupedStream2, aggregator2)
> >>>         .aggregate(initializer, aggregator,
> >>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> >>> sessionMerger).withQueryableStoreName("my-store"));
> >>>
> >>>
> >>>
> >>> public class Count {
> >>>
> >>>     public static class Windowed extends Count {
> >>>         private Windows windows;
> >>>     }
> >>>     public static class SessionWindowed extends Count {
> >>>         private SessionWindows sessionWindows;
> >>>     }
> >>>
> >>>     public static Count count();
> >>>     public static Windowed windowed(Windows windows);
> >>>     public static SessionWindowed sessionWindowed(SessionWindows
> >>> sessionWindows);
> >>>
> >>>     // All withXXX(...) methods.
> >>> }
> >>>
> >>> public class KGroupedStream {
> >>>     public KTable<K, Long> count(Count count);
> >>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> count);
> >>> …
> >>> }
> >>>
> >>>
> >>> Thanks,
> >>> Kyle
> >>>
> >>> From: Guozhang Wang
> >>> Sent: Wednesday, June 28, 2017 7:45 PM
> >>> To: dev@kafka.apache.org
> >>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>
> >>> I played the current proposal a bit with
> https://github.com/dguy/kafka/
> >>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment
> >,
> >>> and here are my observations:
> >>>
> >>> 1. Personally I prefer
> >>>
> >>>     "stream.group(mapper) / stream.groupByKey()"
> >>>
> >>> than
> >>>
> >>>     "stream.group().withKeyMapper(mapper) / stream.group()"
> >>>
> >>> Since 1) withKeyMapper is not enforced programmatically though it is
> not
> >>> "really" optional like others, 2) syntax-wise it reads more natural.
> >>>
> >>> I think it is okay to add the APIs in (
> >>>
> >>>
> >>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> >>> )
> >>> in KGroupedStream.
> >>>
> >>>
> >>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass
> in
> >>> the most-inner state store supplier (e.g. then one whose get() return
> >>> RocksDBStore), or it is supposed to return the most-outer supplier with
> >>> logging / metrics / etc? I think it would be more useful to only
> require
> >>> users pass in the inner state store supplier while specifying caching /
> >>> logging through other APIs.
> >>>
> >>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
> >> are
> >>> allowing users to call other APIs like "withQueryableName" multiple
> time,
> >>> but only call "withStateStoreSupplier" only once in the end. Why is
> that?
> >>>
> >>>
> >>> 3. The current DSL seems to be only for aggregations, what about joins?
> >>>
> >>>
> >>> 4. I think it is okay to keep the "withLogConfig": for the
> >>> StateStoreSupplier it will still be user code specifying the topology
> so
> >> I
> >>> do not see there is a big difference.
> >>>
> >>>
> >>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> >>> windowed state store supplier to enforce typing?
> >>>
> >>>
> >>> Below are minor ones:
> >>>
> >>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> >>>
> >>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >>>
> >>>
> >>>
> >>> Guozhang
> >>>
> >>>
> >>>
> >>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> matthias@confluent.io>
> >>> wrote:
> >>>
> >>>> I see your point about "when to add the processor to the topology".
> >> That
> >>>> is indeed an issue. Not sure it we could allow "updates" to the
> >>> topology...
> >>>>
> >>>> I don't see any problem with having all the withXX() in KTable
> >> interface
> >>>> -- but this might be subjective.
> >>>>
> >>>>
> >>>> However, I don't understand your argument about putting aggregate()
> >>>> after the withXX() -- all the calls to withXX() set optional
> parameters
> >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >>>> be quite confusion for developers.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> >>>>>> I also think that mixing optional parameters with configs is a bad
> >>> idea.
> >>>>>> Have not proposal for this atm but just wanted to mention it. Hope
> >> to
> >>>>>> find some time to come up with something.
> >>>>>>
> >>>>>>
> >>>>> Yes, i don't like the mix of config either. But the only real config
> >>> here
> >>>>> is the logging config - which we don't really need as it can already
> >> be
> >>>>> done via a custom StateStoreSupplier.
> >>>>>
> >>>>>
> >>>>>> What I don't like in the current proposal is the
> >>>>>> .grouped().withKeyMapper() -- the current solution with
> >> .groupBy(...)
> >>>>>> and .groupByKey() seems better. For clarity, we could rename to
> >>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> >>>>>> some better names).
> >>>>>>
> >>>>>>
> >>>>> it could be groupByKey(), groupBy() or something different bt
> >>>>>
> >>>>>
> >>>>>
> >>>>>> The proposed pattern "chains" grouping and aggregation too close
> >>>>>> together. I would rather separate both more than less, ie, do into
> >> the
> >>>>>> opposite direction.
> >>>>>>
> >>>>>> I am also wondering, if we could so something more "fluent". The
> >>> initial
> >>>>>> proposal was like:
> >>>>>>
> >>>>>>>> groupedStream.count()
> >>>>>>>>    .withStoreName("name")
> >>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>    .table()
> >>>>>>
> >>>>>> The .table() statement in the end was kinda alien.
> >>>>>>
> >>>>>
> >>>>> I agree, but then all of the withXXX methods need to be on KTable
> >> which
> >>>> is
> >>>>> worse in my opinion. You also need something that is going to "build"
> >>> the
> >>>>> internal processors and add them to the topology.
> >>>>>
> >>>>>
> >>>>>> The current proposal put the count() into the end -- ie, the
> >> optional
> >>>>>> parameter for count() have to specified on the .grouped() call --
> >> this
> >>>>>> does not seems to be the best way either.
> >>>>>>
> >>>>>>
> >>>>> I actually prefer this method as you are building a grouped stream
> >> that
> >>>> you
> >>>>> will aggregate. So
> >> table.grouped(...).withOptionalStuff().aggregate(..)
> >>>> etc
> >>>>> seems natural to me.
> >>>>>
> >>>>>
> >>>>>> I did not think this through in detail, but can't we just do the
> >>> initial
> >>>>>> proposal with the .table() ?
> >>>>>>
> >>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> >>>>>>
> >>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
> >>> are
> >>>>>> just added to the KTable interface. Or do I miss anything why this
> >>> wont'
> >>>>>> work or any obvious disadvantage?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>> See above.
> >>>>>
> >>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> >> fluent
> >>>>>>> approach, but i think it is slightly nicer.
> >>>>>>> I agree with some of what Eno said about mixing configy stuff in
> >> the
> >>>> DSL,
> >>>>>>> but i think that enabling caching and enabling logging are things
> >>> that
> >>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> >> (even
> >>>>>>> though it is below) as this is actually config and we already have
> >> a
> >>>> way
> >>>>>> of
> >>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
> >>> is
> >>>> a
> >>>>>>> bit of a tedious process for someone that just wants to use the
> >>> default
> >>>>>>> storage engine, but not have caching enabled.
> >>>>>>>
> >>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
> >>> you
> >>>>>>> want to plug in a custom storage engine and you want it to be
> >> logged
> >>>> etc,
> >>>>>>> you would currently need to implement that yourself. Ideally we can
> >>>>>> provide
> >>>>>>> a way where we will wrap the custom store with logging, metrics,
> >>> etc. I
> >>>>>>> need to think about where this fits, it is probably more
> >> appropriate
> >>> on
> >>>>>> the
> >>>>>>> Stores API.
> >>>>>>>
> >>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>>>>>> // count with mapped key
> >>>>>>> final KTable<Long, Long> count = stream.grouped()
> >>>>>>>         .withKeyMapper(keyMapper)
> >>>>>>>         .withKeySerde(Serdes.Long())
> >>>>>>>         .withValueSerde(Serdes.String())
> >>>>>>>         .withQueryableName("my-store")
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // windowed count
> >>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> >> stream.grouped()
> >>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // windowed reduce
> >>>>>>> final Reducer<String> windowedReducer = null;
> >>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> >>>> stream.grouped()
> >>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>         .reduce(windowedReducer);
> >>>>>>>
> >>>>>>> final Aggregator<String, String, Long> aggregator = null;
> >>>>>>> final Initializer<Long> init = null;
> >>>>>>>
> >>>>>>> // aggregate
> >>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> >>>>>>>         .withQueryableName("my-aggregate-store")
> >>>>>>>         .aggregate(aggregator, init, Serdes.Long());
> >>>>>>>
> >>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> >>>> stateStoreSupplier
> >>>>>> = null;
> >>>>>>>
> >>>>>>> // aggregate with custom store
> >>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>>>>>         .aggregate(aggregator, init);
> >>>>>>>
> >>>>>>> // disable caching
> >>>>>>> stream.grouped()
> >>>>>>>         .withQueryableName("name")
> >>>>>>>         .withCachingEnabled(false)
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // disable logging
> >>>>>>> stream.grouped()
> >>>>>>>         .withQueryableName("q")
> >>>>>>>         .withLoggingEnabled(false)
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // override log config
> >>>>>>> final Reducer<String> reducer = null;
> >>>>>>> stream.grouped()
> >>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
> >>> "10"))
> >>>>>>>         .reduce(reducer);
> >>>>>>>
> >>>>>>>
> >>>>>>> If anyone wants to play around with this you can find the code
> >> here:
> >>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>>>>>
> >>>>>>> Note: It won't actually work as most of the methods just return
> >> null.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Damian
> >>>>>>>
> >>>>>>>
> >>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> >> wrote:
> >>>>>>>
> >>>>>>>> Thanks Damian. I think both options have pros and cons. And both
> >> are
> >>>>>> better
> >>>>>>>> than overload abuse.
> >>>>>>>>
> >>>>>>>> The fluent API approach reads better, no mention of builder or
> >> build
> >>>>>>>> anywhere. The main downside is that the method signatures are a
> >>> little
> >>>>>> less
> >>>>>>>> clear. By reading the method signature, one doesn't necessarily
> >>> knows
> >>>>>> what
> >>>>>>>> it returns. Also, one needs to figure out the special method
> >>>> (`table()`
> >>>>>> in
> >>>>>>>> this case) that gives you what you actually care about (`KTable`
> >> in
> >>>> this
> >>>>>>>> case). Not major issues, but worth mentioning while doing the
> >>>>>> comparison.
> >>>>>>>>
> >>>>>>>> The builder approach avoids the issues mentioned above, but it
> >>> doesn't
> >>>>>> read
> >>>>>>>> as well.
> >>>>>>>>
> >>>>>>>> Ismael
> >>>>>>>>
> >>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
> >>>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> I'd like to get a discussion going around some of the API choices
> >>>> we've
> >>>>>>>>> made in the DLS. In particular those that relate to stateful
> >>>> operations
> >>>>>>>>> (though this could expand).
> >>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
> >> i.e,
> >>>>>> there
> >>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> >> noisy
> >>>> and
> >>>>>> i
> >>>>>>>>> feel it is only going to get worse as we add more optional
> >> params.
> >>> In
> >>>>>>>>> particular we've had some requests to be able to turn caching
> >> off,
> >>> or
> >>>>>>>>> change log configs,  on a per operator basis (note this can be
> >> done
> >>>> now
> >>>>>>>> if
> >>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> >>> cumbersome).
> >>>>>>>>>
> >>>>>>>>> So this is a bit of an open question. How can we change the DSL
> >>>>>> overloads
> >>>>>>>>> so that it flows, is simple to use and understand, and is easily
> >>>>>> extended
> >>>>>>>>> in the future?
> >>>>>>>>>
> >>>>>>>>> One option would be to use a fluent API approach for providing
> >> the
> >>>>>>>> optional
> >>>>>>>>> params, so something like this:
> >>>>>>>>>
> >>>>>>>>> groupedStream.count()
> >>>>>>>>>    .withStoreName("name")
> >>>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>>    .table()
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Another option would be to provide a Builder to the count method,
> >>> so
> >>>> it
> >>>>>>>>> would look something like this:
> >>>>>>>>> groupedStream.count(new
> >>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>>>>>
> >>>>>>>>> Another option is to say: Hey we don't need this, what are you on
> >>>>>> about!
> >>>>>>>>>
> >>>>>>>>> The above has focussed on state store related overloads, but the
> >>> same
> >>>>>>>> ideas
> >>>>>>>>> could  be applied to joins etc, where we presently have many join
> >>>>>> methods
> >>>>>>>>> and many overloads.
> >>>>>>>>>
> >>>>>>>>> Anyway, i look forward to hearing your opinions.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>>
> >>
> >>
> >
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks Matthias

On Fri, 30 Jun 2017 at 08:05 Matthias J. Sax <ma...@confluent.io> wrote:

> I am just catching up on this thread, so sorry for the long email in
> advance... Also, it's to some extend a dump of thoughts and not always a
> clear proposal. Still need to think about this in more detail. But maybe
> it helps other to get new ideas :)
>
>
> >> However, I don't understand your argument about putting aggregate()
> >> after the withXX() -- all the calls to withXX() set optional parameters
> >> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >> be quite confusion for developers.
> >>
> >>
> > I see what you are saying, but the grouped stream is effectively a no-op
> > until you call one of the aggregate/count/reduce etc functions. So the
> > optional params are ones that are applicable to any of the operations you
> > can perform on this grouped stream. Then the final
> > count()/reduce()/aggregate() call has any of the params that are
> > required/specific to that function.
> >
>
> I understand your argument, but you don't share the conclusion. If we
> need a "final/terminal" call, the better way might be
>
> .groupBy().count().withXX().build()
>
> (with a better name for build() though)
>
>
The point is that all the other calls, i.e,withBlah, windowed, etc apply
too all the aggregate functions. The terminal call being the actual type of
aggregation you want to do. I personally find this more natural than
groupBy().count().withBlah().build()


> > groupedStream.count(/** non windowed count**/)
> > groupedStream.windowed(TimeWindows.of(10L)).count(...)
> > groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>
> I like this. However, I don't see a reason to have windowed() and
> sessionWindowed(). We should have one top-level `Windows` interface that
> both `TimeWindows` and `SessionWindows` implement and just have a single
> windowed() method that accepts all `Windows`. (I did not like the
> separation of `SessionWindows` in the first place, and this seems to be
> an opportunity to clean this up. It was hard to change when we
> introduced session windows)
>

Yes - true we should look into that.


>
> Btw: we do you the imperative groupBy() and groupByKey(), and thus we
> might also want to use windowBy() (instead of windowed()). Not sure how
> important this is, but it seems to be inconsistent otherwise.
>
>
Makes sense


>
> About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
> defining an inner/left/outer join is not an optional argument but a
> first class concept and should have a proper representation in the API
> (like the current methods join(), leftJoin, outerJoin()).
>
>
Yep, i did originally have it as a required param and maybe that is what we
go with. It could have a default, but maybe that is confusing.



> About the two join API proposals, the second one has too much boiler
> plate code for my taste. Also, the actual join() operator has only one
> argument what is weird to me, as in my thinking process, the main
> operator call, should have one parameter per mandatory argument but your
> proposal put the mandatory arguments into Joins.streamStreamJoin() call.
> This is far from intuitive IMHO.
>
>
This is the builder pattern, you only need one param as the builder has
captured all of the required and optional arguments.


> The first join proposal also seems to align better with the pattern
> suggested for aggregations and having the same pattern for all operators
> is important (as you stated already).
>
>
This is why i offered two alternatives as i started out with. 1 is the
builder pattern, the other is the more fluent pattern.


>
>
> Coming back to the config vs optional parameter. What about having a
> method withConfig[s](...) that allow to put in the configuration?
>
>
Sure, it is currently called withLogConfig() as that is the only thing that
is really config.


> This also raises the question if until() is a windows property?
> Actually, until() seems to be a configuration parameter and thus, should
> not not have it's own method.
>
>
Hmmm, i don't agree. Until is a property of the window. It is going to be
potentially different for every window operation you do in a streams app.


>
>
> Browsing throw your example DSL branch, I also saw this one:
>
> > final KTable<Windowed<String>, Long> windowed>
>  groupedStream.counting()
> >                  .windowed(TimeWindows.of(10L).until(10))
> >                  .table();
>
> This is an interesting idea, and it remind my on some feedback about "I
> wanted to count a stream, but there was no count() method -- I first
> needed to figure out, that I need to group the stream first to be able
> to count it. It does make sense in hindsight but was not obvious in the
> beginning". Thus, carrying out this thought, we could also do the
> following:
>
> stream.count().groupedBy().windowedBy().table();
>
> -> Note, I use "grouped" and "windowed" instead of imperative here, as
> it comes after the count()
>
> This would be more consistent than your proposal (that has grouping
> before but windowing after count()). It might even allow us to enrich
> the API with a some syntactic sugar like `stream.count().table()` to get
> the overall count of all records (this would obviously not scale, but we
> could support it -- if not now, maybe later).
>
>
I guess i'd prefer
stream.groupBy().windowBy().count()
stream.groupBy().windowBy().reduce()
stream.groupBy().count()

As i said above, everything that happens before the final aggregate call
can be applied to any of them. So it makes sense to me to do those things
ahead of the final aggregate call.


> Last about builder pattern. I am convinced that we need some "terminal"
> operator/method that tells us when to add the processor to the topology.
> But I don't see the need for a plain builder pattern that feels alien to
> me (see my argument about the second join proposal). Using .stream() /
> .table() as use in many examples might work. But maybe a more generic
> name that we can use in all places like build() or apply() might also be
> an option.
>
>
Sure, a generic name might be ok.




>
> -Matthias
>
>
>
> On 6/29/17 7:37 AM, Damian Guy wrote:
> > Thanks Kyle.
> >
> > On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> > wrote:
> >
> >> Hi Damian,
> >>
> >>>>>> When trying to program in the fluent API that has been discussed
> most
> >> it
> >>>>>> feels difficult to know when you will actually get an object you can
> >> reuse.
> >>>>>> What if I make one KGroupedStream that I want to reuse, is it legal
> to
> >>>>>> reuse it or does this approach expect you to call grouped each time?
> >>
> >>>> I'd anticipate that once you have a KGroupedStream you can re-use it
> as
> >> you
> >>>> can today.
> >>
> >> You said it yourself in another post that the grouped stream is
> >> effectively a no-op until a count, reduce, or aggregate. The way I see
> it
> >> you wouldn’t be able to reuse anything except KStreams and KTables,
> because
> >> most of this fluent api would continue returning this (this being the
> >> builder object currently being manipulated).
> >
> > So, if you ever store a reference to anything but KStreams and KTables
> and
> >> you use it in two different ways then its possible you make conflicting
> >> withXXX() calls on the same builder.
> >>
> >>
> > No necessarily true. It could return a new instance of the builder, i.e.,
> > the builders being immutable. So if you held a reference to the builder
> it
> > would always be the same as it was when it was created.
> >
> >
> >> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> >> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> >> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
> >>
> >> I’ll admit that this shouldn’t happen but some user is going to do it
> >> eventually…
> >> Depending on implementation uses of groupedStreamWithDefaultSerdes would
> >> most likely be equivalent to the version withDeclaredSerdes. One work
> >> around would be to always make copies of the config objects you are
> >> building, but this approach has its own problem because now we have to
> >> identify which configs are equivalent so we don’t create repeated
> >> processors.
> >>
> >> The point of this long winded example is that we always have to be
> >> thinking about all of the possible ways it could be misused by a user
> >> (causing them to see hard to diagnose problems).
> >>
> >
> > Exactly! That is the point of the discussion really.
> >
> >
> >>
> >> In my attempt at a couple methods with builders I feel that I could
> >> confidently say the user couldn’t really mess it up.
> >>> // Count
> >>> KTable<String, Long> count =
> >>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >> The kGroupedStream is reusable and if they attempted to reuse the Count
> >> for some reason it would throw an error message saying that a store
> named
> >> “my-store” already exists.
> >>
> >>
> > Yes i agree and i think using builders is my preferred pattern.
> >
> > Cheers,
> > Damian
> >
> >
> >> Thanks,
> >> Kyle
> >>
> >> From: Damian Guy
> >> Sent: Thursday, June 29, 2017 3:59 AM
> >> To: dev@kafka.apache.org
> >> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>
> >> Hi Kyle,
> >>
> >> Thanks for your input. Really appreciated.
> >>
> >> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
> >> wrote:
> >>
> >>> I like more of a builder pattern even though others have voiced against
> >>> it. The reason I like it is because it makes it clear to the user that
> a
> >>> call to KGroupedStream#count will return a KTable not some intermediate
> >>> class that I need to undetstand.
> >>>
> >>
> >> Yes, that makes sense.
> >>
> >>
> >>> When trying to program in the fluent API that has been discussed most
> it
> >>> feels difficult to know when you will actually get an object you can
> >> reuse.
> >>> What if I make one KGroupedStream that I want to reuse, is it legal to
> >>> reuse it or does this approach expect you to call grouped each time?
> >>
> >>
> >> I'd anticipate that once you have a KGroupedStream you can re-use it as
> you
> >> can today.
> >>
> >>
> >>> This question doesn’t pop into my head at all in the builder pattern I
> >>> assume I can reuse everything.
> >>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan
> of
> >>> the grouped.
> >>>
> >>> Yes, grouped() was more for demonstration and because groupBy() and
> >> groupByKey() were taken! So i'd imagine the api would actually want to
> be
> >> groupByKey(/** no required args***/).withOptionalArg() and
> >> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all
> depends
> >> on maintaining backward compatibility.
> >>
> >>
> >>> Unfortunately, the below approach would require atleast 2 (probably 3)
> >>> overloads (one for returning a KTable and one for returning a KTable
> with
> >>> Windowed Key, probably would want to split windowed and sessionwindowed
> >> for
> >>> ease of implementation) of each count, reduce, and aggregate.
> >>> Obviously not exhaustive but enough for you to get the picture. Count,
> >>> Reduce, and Aggregate supply 3 static methods to initialize the
> builder:
> >>> // Count
> >>> KTable<String, Long> count =
> >>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >>>
> >>> // Windowed Count
> >>> KTable<Windowed<String>, Long> windowedCount =
> >>>
> >>
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >>>
> >>> // Session Count
> >>> KTable<Windowed<String>, Long> sessionCount =
> >>>
> >>
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >>>
> >>>
> >> Above and below, i think i'd prefer it to be:
> >> groupedStream.count(/** non windowed count**/)
> >> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> >> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
> >>
> >>
> >>
> >>
> >>> // Reduce
> >>> Reducer<Long> reducer;
> >>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> >>> Reduce.reduce().withQueryableStoreName("my-store"));
> >>>
> >>> // Aggregate Windowed with Custom Store
> >>> Initializer<String> initializer;
> >>> Aggregator<String, Long, String> aggregator;
> >>> KTable<Windowed<String>, String> aggregate =
> >>> groupedStream.aggregate(initializer, aggregator,
> >>>
> >>
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >>>
> >>> // Cogroup SessionWindowed
> >>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
> >>>         .cogroup(groupedStream2, aggregator2)
> >>>         .aggregate(initializer, aggregator,
> >>> Aggregate.sessionWindowed(SessionWindows.with(10L),
> >>> sessionMerger).withQueryableStoreName("my-store"));
> >>>
> >>>
> >>>
> >>> public class Count {
> >>>
> >>>     public static class Windowed extends Count {
> >>>         private Windows windows;
> >>>     }
> >>>     public static class SessionWindowed extends Count {
> >>>         private SessionWindows sessionWindows;
> >>>     }
> >>>
> >>>     public static Count count();
> >>>     public static Windowed windowed(Windows windows);
> >>>     public static SessionWindowed sessionWindowed(SessionWindows
> >>> sessionWindows);
> >>>
> >>>     // All withXXX(...) methods.
> >>> }
> >>>
> >>> public class KGroupedStream {
> >>>     public KTable<K, Long> count(Count count);
> >>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed
> count);
> >>> …
> >>> }
> >>>
> >>>
> >>> Thanks,
> >>> Kyle
> >>>
> >>> From: Guozhang Wang
> >>> Sent: Wednesday, June 28, 2017 7:45 PM
> >>> To: dev@kafka.apache.org
> >>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >>>
> >>> I played the current proposal a bit with
> https://github.com/dguy/kafka/
> >>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment
> >,
> >>> and here are my observations:
> >>>
> >>> 1. Personally I prefer
> >>>
> >>>     "stream.group(mapper) / stream.groupByKey()"
> >>>
> >>> than
> >>>
> >>>     "stream.group().withKeyMapper(mapper) / stream.group()"
> >>>
> >>> Since 1) withKeyMapper is not enforced programmatically though it is
> not
> >>> "really" optional like others, 2) syntax-wise it reads more natural.
> >>>
> >>> I think it is okay to add the APIs in (
> >>>
> >>>
> >>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> >>> )
> >>> in KGroupedStream.
> >>>
> >>>
> >>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass
> in
> >>> the most-inner state store supplier (e.g. then one whose get() return
> >>> RocksDBStore), or it is supposed to return the most-outer supplier with
> >>> logging / metrics / etc? I think it would be more useful to only
> require
> >>> users pass in the inner state store supplier while specifying caching /
> >>> logging through other APIs.
> >>>
> >>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
> >> are
> >>> allowing users to call other APIs like "withQueryableName" multiple
> time,
> >>> but only call "withStateStoreSupplier" only once in the end. Why is
> that?
> >>>
> >>>
> >>> 3. The current DSL seems to be only for aggregations, what about joins?
> >>>
> >>>
> >>> 4. I think it is okay to keep the "withLogConfig": for the
> >>> StateStoreSupplier it will still be user code specifying the topology
> so
> >> I
> >>> do not see there is a big difference.
> >>>
> >>>
> >>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> >>> windowed state store supplier to enforce typing?
> >>>
> >>>
> >>> Below are minor ones:
> >>>
> >>> 6. "withQueryableName": maybe better "withQueryableStateName"?
> >>>
> >>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >>>
> >>>
> >>>
> >>> Guozhang
> >>>
> >>>
> >>>
> >>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <
> matthias@confluent.io>
> >>> wrote:
> >>>
> >>>> I see your point about "when to add the processor to the topology".
> >> That
> >>>> is indeed an issue. Not sure it we could allow "updates" to the
> >>> topology...
> >>>>
> >>>> I don't see any problem with having all the withXX() in KTable
> >> interface
> >>>> -- but this might be subjective.
> >>>>
> >>>>
> >>>> However, I don't understand your argument about putting aggregate()
> >>>> after the withXX() -- all the calls to withXX() set optional
> parameters
> >>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> >>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> >>>> be quite confusion for developers.
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 6/28/17 2:55 AM, Damian Guy wrote:
> >>>>>> I also think that mixing optional parameters with configs is a bad
> >>> idea.
> >>>>>> Have not proposal for this atm but just wanted to mention it. Hope
> >> to
> >>>>>> find some time to come up with something.
> >>>>>>
> >>>>>>
> >>>>> Yes, i don't like the mix of config either. But the only real config
> >>> here
> >>>>> is the logging config - which we don't really need as it can already
> >> be
> >>>>> done via a custom StateStoreSupplier.
> >>>>>
> >>>>>
> >>>>>> What I don't like in the current proposal is the
> >>>>>> .grouped().withKeyMapper() -- the current solution with
> >> .groupBy(...)
> >>>>>> and .groupByKey() seems better. For clarity, we could rename to
> >>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> >>>>>> some better names).
> >>>>>>
> >>>>>>
> >>>>> it could be groupByKey(), groupBy() or something different bt
> >>>>>
> >>>>>
> >>>>>
> >>>>>> The proposed pattern "chains" grouping and aggregation too close
> >>>>>> together. I would rather separate both more than less, ie, do into
> >> the
> >>>>>> opposite direction.
> >>>>>>
> >>>>>> I am also wondering, if we could so something more "fluent". The
> >>> initial
> >>>>>> proposal was like:
> >>>>>>
> >>>>>>>> groupedStream.count()
> >>>>>>>>    .withStoreName("name")
> >>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>    .table()
> >>>>>>
> >>>>>> The .table() statement in the end was kinda alien.
> >>>>>>
> >>>>>
> >>>>> I agree, but then all of the withXXX methods need to be on KTable
> >> which
> >>>> is
> >>>>> worse in my opinion. You also need something that is going to "build"
> >>> the
> >>>>> internal processors and add them to the topology.
> >>>>>
> >>>>>
> >>>>>> The current proposal put the count() into the end -- ie, the
> >> optional
> >>>>>> parameter for count() have to specified on the .grouped() call --
> >> this
> >>>>>> does not seems to be the best way either.
> >>>>>>
> >>>>>>
> >>>>> I actually prefer this method as you are building a grouped stream
> >> that
> >>>> you
> >>>>> will aggregate. So
> >> table.grouped(...).withOptionalStuff().aggregate(..)
> >>>> etc
> >>>>> seems natural to me.
> >>>>>
> >>>>>
> >>>>>> I did not think this through in detail, but can't we just do the
> >>> initial
> >>>>>> proposal with the .table() ?
> >>>>>>
> >>>>>> groupedStream.count().withStoreName("name").mapValues(...)
> >>>>>>
> >>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
> >>> are
> >>>>>> just added to the KTable interface. Or do I miss anything why this
> >>> wont'
> >>>>>> work or any obvious disadvantage?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>> See above.
> >>>>>
> >>>>>
> >>>>>>
> >>>>>> -Matthias
> >>>>>>
> >>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>>>>>> Thanks everyone. My latest attempt is below. It builds on the
> >> fluent
> >>>>>>> approach, but i think it is slightly nicer.
> >>>>>>> I agree with some of what Eno said about mixing configy stuff in
> >> the
> >>>> DSL,
> >>>>>>> but i think that enabling caching and enabling logging are things
> >>> that
> >>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
> >> (even
> >>>>>>> though it is below) as this is actually config and we already have
> >> a
> >>>> way
> >>>>>> of
> >>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
> >>> is
> >>>> a
> >>>>>>> bit of a tedious process for someone that just wants to use the
> >>> default
> >>>>>>> storage engine, but not have caching enabled.
> >>>>>>>
> >>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
> >>> you
> >>>>>>> want to plug in a custom storage engine and you want it to be
> >> logged
> >>>> etc,
> >>>>>>> you would currently need to implement that yourself. Ideally we can
> >>>>>> provide
> >>>>>>> a way where we will wrap the custom store with logging, metrics,
> >>> etc. I
> >>>>>>> need to think about where this fits, it is probably more
> >> appropriate
> >>> on
> >>>>>> the
> >>>>>>> Stores API.
> >>>>>>>
> >>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>>>>>> // count with mapped key
> >>>>>>> final KTable<Long, Long> count = stream.grouped()
> >>>>>>>         .withKeyMapper(keyMapper)
> >>>>>>>         .withKeySerde(Serdes.Long())
> >>>>>>>         .withValueSerde(Serdes.String())
> >>>>>>>         .withQueryableName("my-store")
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // windowed count
> >>>>>>> final KTable<Windowed<String>, Long> windowedCount =
> >> stream.grouped()
> >>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // windowed reduce
> >>>>>>> final Reducer<String> windowedReducer = null;
> >>>>>>> final KTable<Windowed<String>, String> windowedReduce =
> >>>> stream.grouped()
> >>>>>>>         .withQueryableName("my-window-store")
> >>>>>>>         .windowed(TimeWindows.of(10L).until(10))
> >>>>>>>         .reduce(windowedReducer);
> >>>>>>>
> >>>>>>> final Aggregator<String, String, Long> aggregator = null;
> >>>>>>> final Initializer<Long> init = null;
> >>>>>>>
> >>>>>>> // aggregate
> >>>>>>> final KTable<String, Long> aggregate = stream.grouped()
> >>>>>>>         .withQueryableName("my-aggregate-store")
> >>>>>>>         .aggregate(aggregator, init, Serdes.Long());
> >>>>>>>
> >>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
> >>>> stateStoreSupplier
> >>>>>> = null;
> >>>>>>>
> >>>>>>> // aggregate with custom store
> >>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>>>>>         .aggregate(aggregator, init);
> >>>>>>>
> >>>>>>> // disable caching
> >>>>>>> stream.grouped()
> >>>>>>>         .withQueryableName("name")
> >>>>>>>         .withCachingEnabled(false)
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // disable logging
> >>>>>>> stream.grouped()
> >>>>>>>         .withQueryableName("q")
> >>>>>>>         .withLoggingEnabled(false)
> >>>>>>>         .count();
> >>>>>>>
> >>>>>>> // override log config
> >>>>>>> final Reducer<String> reducer = null;
> >>>>>>> stream.grouped()
> >>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
> >>> "10"))
> >>>>>>>         .reduce(reducer);
> >>>>>>>
> >>>>>>>
> >>>>>>> If anyone wants to play around with this you can find the code
> >> here:
> >>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>>>>>
> >>>>>>> Note: It won't actually work as most of the methods just return
> >> null.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> Damian
> >>>>>>>
> >>>>>>>
> >>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> >> wrote:
> >>>>>>>
> >>>>>>>> Thanks Damian. I think both options have pros and cons. And both
> >> are
> >>>>>> better
> >>>>>>>> than overload abuse.
> >>>>>>>>
> >>>>>>>> The fluent API approach reads better, no mention of builder or
> >> build
> >>>>>>>> anywhere. The main downside is that the method signatures are a
> >>> little
> >>>>>> less
> >>>>>>>> clear. By reading the method signature, one doesn't necessarily
> >>> knows
> >>>>>> what
> >>>>>>>> it returns. Also, one needs to figure out the special method
> >>>> (`table()`
> >>>>>> in
> >>>>>>>> this case) that gives you what you actually care about (`KTable`
> >> in
> >>>> this
> >>>>>>>> case). Not major issues, but worth mentioning while doing the
> >>>>>> comparison.
> >>>>>>>>
> >>>>>>>> The builder approach avoids the issues mentioned above, but it
> >>> doesn't
> >>>>>> read
> >>>>>>>> as well.
> >>>>>>>>
> >>>>>>>> Ismael
> >>>>>>>>
> >>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
> >>>
> >>>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Hi,
> >>>>>>>>>
> >>>>>>>>> I'd like to get a discussion going around some of the API choices
> >>>> we've
> >>>>>>>>> made in the DLS. In particular those that relate to stateful
> >>>> operations
> >>>>>>>>> (though this could expand).
> >>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
> >> i.e,
> >>>>>> there
> >>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> >> noisy
> >>>> and
> >>>>>> i
> >>>>>>>>> feel it is only going to get worse as we add more optional
> >> params.
> >>> In
> >>>>>>>>> particular we've had some requests to be able to turn caching
> >> off,
> >>> or
> >>>>>>>>> change log configs,  on a per operator basis (note this can be
> >> done
> >>>> now
> >>>>>>>> if
> >>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
> >>> cumbersome).
> >>>>>>>>>
> >>>>>>>>> So this is a bit of an open question. How can we change the DSL
> >>>>>> overloads
> >>>>>>>>> so that it flows, is simple to use and understand, and is easily
> >>>>>> extended
> >>>>>>>>> in the future?
> >>>>>>>>>
> >>>>>>>>> One option would be to use a fluent API approach for providing
> >> the
> >>>>>>>> optional
> >>>>>>>>> params, so something like this:
> >>>>>>>>>
> >>>>>>>>> groupedStream.count()
> >>>>>>>>>    .withStoreName("name")
> >>>>>>>>>    .withCachingEnabled(false)
> >>>>>>>>>    .withLoggingEnabled(config)
> >>>>>>>>>    .table()
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>>
> >>>>>>>>> Another option would be to provide a Builder to the count method,
> >>> so
> >>>> it
> >>>>>>>>> would look something like this:
> >>>>>>>>> groupedStream.count(new
> >>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>>>>>
> >>>>>>>>> Another option is to say: Hey we don't need this, what are you on
> >>>>>> about!
> >>>>>>>>>
> >>>>>>>>> The above has focussed on state store related overloads, but the
> >>> same
> >>>>>>>> ideas
> >>>>>>>>> could  be applied to joins etc, where we presently have many join
> >>>>>> methods
> >>>>>>>>> and many overloads.
> >>>>>>>>>
> >>>>>>>>> Anyway, i look forward to hearing your opinions.
> >>>>>>>>>
> >>>>>>>>> Thanks,
> >>>>>>>>> Damian
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>>
> >>
> >>
> >
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I am just catching up on this thread, so sorry for the long email in
advance... Also, it's to some extend a dump of thoughts and not always a
clear proposal. Still need to think about this in more detail. But maybe
it helps other to get new ideas :)


>> However, I don't understand your argument about putting aggregate()
>> after the withXX() -- all the calls to withXX() set optional parameters
>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>> be quite confusion for developers.
>>
>>
> I see what you are saying, but the grouped stream is effectively a no-op
> until you call one of the aggregate/count/reduce etc functions. So the
> optional params are ones that are applicable to any of the operations you
> can perform on this grouped stream. Then the final
> count()/reduce()/aggregate() call has any of the params that are
> required/specific to that function.
> 

I understand your argument, but you don't share the conclusion. If we
need a "final/terminal" call, the better way might be

.groupBy().count().withXX().build()

(with a better name for build() though)


> groupedStream.count(/** non windowed count**/)
> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)


I like this. However, I don't see a reason to have windowed() and
sessionWindowed(). We should have one top-level `Windows` interface that
both `TimeWindows` and `SessionWindows` implement and just have a single
windowed() method that accepts all `Windows`. (I did not like the
separation of `SessionWindows` in the first place, and this seems to be
an opportunity to clean this up. It was hard to change when we
introduced session windows)

Btw: we do you the imperative groupBy() and groupByKey(), and thus we
might also want to use windowBy() (instead of windowed()). Not sure how
important this is, but it seems to be inconsistent otherwise.


About joins:  I don't like .withJoinType(JoinType.LEFT) at all. I think,
defining an inner/left/outer join is not an optional argument but a
first class concept and should have a proper representation in the API
(like the current methods join(), leftJoin, outerJoin()).

About the two join API proposals, the second one has too much boiler
plate code for my taste. Also, the actual join() operator has only one
argument what is weird to me, as in my thinking process, the main
operator call, should have one parameter per mandatory argument but your
proposal put the mandatory arguments into Joins.streamStreamJoin() call.
This is far from intuitive IMHO.

The first join proposal also seems to align better with the pattern
suggested for aggregations and having the same pattern for all operators
is important (as you stated already).



Coming back to the config vs optional parameter. What about having a
method withConfig[s](...) that allow to put in the configuration?

This also raises the question if until() is a windows property?
Actually, until() seems to be a configuration parameter and thus, should
not not have it's own method.



Browsing throw your example DSL branch, I also saw this one:

> final KTable<Windowed<String>, Long> windowed>     groupedStream.counting()
>                  .windowed(TimeWindows.of(10L).until(10))
>                  .table();

This is an interesting idea, and it remind my on some feedback about "I
wanted to count a stream, but there was no count() method -- I first
needed to figure out, that I need to group the stream first to be able
to count it. It does make sense in hindsight but was not obvious in the
beginning". Thus, carrying out this thought, we could also do the following:

stream.count().groupedBy().windowedBy().table();

-> Note, I use "grouped" and "windowed" instead of imperative here, as
it comes after the count()

This would be more consistent than your proposal (that has grouping
before but windowing after count()). It might even allow us to enrich
the API with a some syntactic sugar like `stream.count().table()` to get
the overall count of all records (this would obviously not scale, but we
could support it -- if not now, maybe later).


Last about builder pattern. I am convinced that we need some "terminal"
operator/method that tells us when to add the processor to the topology.
But I don't see the need for a plain builder pattern that feels alien to
me (see my argument about the second join proposal). Using .stream() /
.table() as use in many examples might work. But maybe a more generic
name that we can use in all places like build() or apply() might also be
an option.



-Matthias



On 6/29/17 7:37 AM, Damian Guy wrote:
> Thanks Kyle.
> 
> On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
> wrote:
> 
>> Hi Damian,
>>
>>>>>> When trying to program in the fluent API that has been discussed most
>> it
>>>>>> feels difficult to know when you will actually get an object you can
>> reuse.
>>>>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>>>>> reuse it or does this approach expect you to call grouped each time?
>>
>>>> I'd anticipate that once you have a KGroupedStream you can re-use it as
>> you
>>>> can today.
>>
>> You said it yourself in another post that the grouped stream is
>> effectively a no-op until a count, reduce, or aggregate. The way I see it
>> you wouldn’t be able to reuse anything except KStreams and KTables, because
>> most of this fluent api would continue returning this (this being the
>> builder object currently being manipulated).
> 
> So, if you ever store a reference to anything but KStreams and KTables and
>> you use it in two different ways then its possible you make conflicting
>> withXXX() calls on the same builder.
>>
>>
> No necessarily true. It could return a new instance of the builder, i.e.,
> the builders being immutable. So if you held a reference to the builder it
> would always be the same as it was when it was created.
> 
> 
>> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
>> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
>> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>>
>> I’ll admit that this shouldn’t happen but some user is going to do it
>> eventually…
>> Depending on implementation uses of groupedStreamWithDefaultSerdes would
>> most likely be equivalent to the version withDeclaredSerdes. One work
>> around would be to always make copies of the config objects you are
>> building, but this approach has its own problem because now we have to
>> identify which configs are equivalent so we don’t create repeated
>> processors.
>>
>> The point of this long winded example is that we always have to be
>> thinking about all of the possible ways it could be misused by a user
>> (causing them to see hard to diagnose problems).
>>
> 
> Exactly! That is the point of the discussion really.
> 
> 
>>
>> In my attempt at a couple methods with builders I feel that I could
>> confidently say the user couldn’t really mess it up.
>>> // Count
>>> KTable<String, Long> count =
>>> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
>> The kGroupedStream is reusable and if they attempted to reuse the Count
>> for some reason it would throw an error message saying that a store named
>> “my-store” already exists.
>>
>>
> Yes i agree and i think using builders is my preferred pattern.
> 
> Cheers,
> Damian
> 
> 
>> Thanks,
>> Kyle
>>
>> From: Damian Guy
>> Sent: Thursday, June 29, 2017 3:59 AM
>> To: dev@kafka.apache.org
>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>
>> Hi Kyle,
>>
>> Thanks for your input. Really appreciated.
>>
>> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
>> wrote:
>>
>>> I like more of a builder pattern even though others have voiced against
>>> it. The reason I like it is because it makes it clear to the user that a
>>> call to KGroupedStream#count will return a KTable not some intermediate
>>> class that I need to undetstand.
>>>
>>
>> Yes, that makes sense.
>>
>>
>>> When trying to program in the fluent API that has been discussed most it
>>> feels difficult to know when you will actually get an object you can
>> reuse.
>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>> reuse it or does this approach expect you to call grouped each time?
>>
>>
>> I'd anticipate that once you have a KGroupedStream you can re-use it as you
>> can today.
>>
>>
>>> This question doesn’t pop into my head at all in the builder pattern I
>>> assume I can reuse everything.
>>> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan of
>>> the grouped.
>>>
>>> Yes, grouped() was more for demonstration and because groupBy() and
>> groupByKey() were taken! So i'd imagine the api would actually want to be
>> groupByKey(/** no required args***/).withOptionalArg() and
>> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all depends
>> on maintaining backward compatibility.
>>
>>
>>> Unfortunately, the below approach would require atleast 2 (probably 3)
>>> overloads (one for returning a KTable and one for returning a KTable with
>>> Windowed Key, probably would want to split windowed and sessionwindowed
>> for
>>> ease of implementation) of each count, reduce, and aggregate.
>>> Obviously not exhaustive but enough for you to get the picture. Count,
>>> Reduce, and Aggregate supply 3 static methods to initialize the builder:
>>> // Count
>>> KTable<String, Long> count =
>>> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>>>
>>> // Windowed Count
>>> KTable<Windowed<String>, Long> windowedCount =
>>>
>> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>>>
>>> // Session Count
>>> KTable<Windowed<String>, Long> sessionCount =
>>>
>> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>>>
>>>
>> Above and below, i think i'd prefer it to be:
>> groupedStream.count(/** non windowed count**/)
>> groupedStream.windowed(TimeWindows.of(10L)).count(...)
>> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>>
>>
>>
>>
>>> // Reduce
>>> Reducer<Long> reducer;
>>> KTable<String, Long> reduce = groupedStream.reduce(reducer,
>>> Reduce.reduce().withQueryableStoreName("my-store"));
>>>
>>> // Aggregate Windowed with Custom Store
>>> Initializer<String> initializer;
>>> Aggregator<String, Long, String> aggregator;
>>> KTable<Windowed<String>, String> aggregate =
>>> groupedStream.aggregate(initializer, aggregator,
>>>
>> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>>>
>>> // Cogroup SessionWindowed
>>> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>>>         .cogroup(groupedStream2, aggregator2)
>>>         .aggregate(initializer, aggregator,
>>> Aggregate.sessionWindowed(SessionWindows.with(10L),
>>> sessionMerger).withQueryableStoreName("my-store"));
>>>
>>>
>>>
>>> public class Count {
>>>
>>>     public static class Windowed extends Count {
>>>         private Windows windows;
>>>     }
>>>     public static class SessionWindowed extends Count {
>>>         private SessionWindows sessionWindows;
>>>     }
>>>
>>>     public static Count count();
>>>     public static Windowed windowed(Windows windows);
>>>     public static SessionWindowed sessionWindowed(SessionWindows
>>> sessionWindows);
>>>
>>>     // All withXXX(...) methods.
>>> }
>>>
>>> public class KGroupedStream {
>>>     public KTable<K, Long> count(Count count);
>>>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
>>>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed count);
>>> …
>>> }
>>>
>>>
>>> Thanks,
>>> Kyle
>>>
>>> From: Guozhang Wang
>>> Sent: Wednesday, June 28, 2017 7:45 PM
>>> To: dev@kafka.apache.org
>>> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>>>
>>> I played the current proposal a bit with https://github.com/dguy/kafka/
>>> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment>,
>>> and here are my observations:
>>>
>>> 1. Personally I prefer
>>>
>>>     "stream.group(mapper) / stream.groupByKey()"
>>>
>>> than
>>>
>>>     "stream.group().withKeyMapper(mapper) / stream.group()"
>>>
>>> Since 1) withKeyMapper is not enforced programmatically though it is not
>>> "really" optional like others, 2) syntax-wise it reads more natural.
>>>
>>> I think it is okay to add the APIs in (
>>>
>>>
>> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
>>> )
>>> in KGroupedStream.
>>>
>>>
>>> 2. For the "withStateStoreSupplier" API, are the user supposed to pass in
>>> the most-inner state store supplier (e.g. then one whose get() return
>>> RocksDBStore), or it is supposed to return the most-outer supplier with
>>> logging / metrics / etc? I think it would be more useful to only require
>>> users pass in the inner state store supplier while specifying caching /
>>> logging through other APIs.
>>>
>>> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
>> are
>>> allowing users to call other APIs like "withQueryableName" multiple time,
>>> but only call "withStateStoreSupplier" only once in the end. Why is that?
>>>
>>>
>>> 3. The current DSL seems to be only for aggregations, what about joins?
>>>
>>>
>>> 4. I think it is okay to keep the "withLogConfig": for the
>>> StateStoreSupplier it will still be user code specifying the topology so
>> I
>>> do not see there is a big difference.
>>>
>>>
>>> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
>>> windowed state store supplier to enforce typing?
>>>
>>>
>>> Below are minor ones:
>>>
>>> 6. "withQueryableName": maybe better "withQueryableStateName"?
>>>
>>> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>>>
>>>
>>>
>>> Guozhang
>>>
>>>
>>>
>>> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> I see your point about "when to add the processor to the topology".
>> That
>>>> is indeed an issue. Not sure it we could allow "updates" to the
>>> topology...
>>>>
>>>> I don't see any problem with having all the withXX() in KTable
>> interface
>>>> -- but this might be subjective.
>>>>
>>>>
>>>> However, I don't understand your argument about putting aggregate()
>>>> after the withXX() -- all the calls to withXX() set optional parameters
>>>> for aggregate() and not for groupBy() -- but a groupBy().withXX()
>>>> indicates that the withXX() belongs to the groupBy(). IMHO, this might
>>>> be quite confusion for developers.
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 6/28/17 2:55 AM, Damian Guy wrote:
>>>>>> I also think that mixing optional parameters with configs is a bad
>>> idea.
>>>>>> Have not proposal for this atm but just wanted to mention it. Hope
>> to
>>>>>> find some time to come up with something.
>>>>>>
>>>>>>
>>>>> Yes, i don't like the mix of config either. But the only real config
>>> here
>>>>> is the logging config - which we don't really need as it can already
>> be
>>>>> done via a custom StateStoreSupplier.
>>>>>
>>>>>
>>>>>> What I don't like in the current proposal is the
>>>>>> .grouped().withKeyMapper() -- the current solution with
>> .groupBy(...)
>>>>>> and .groupByKey() seems better. For clarity, we could rename to
>>>>>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>>>>>> some better names).
>>>>>>
>>>>>>
>>>>> it could be groupByKey(), groupBy() or something different bt
>>>>>
>>>>>
>>>>>
>>>>>> The proposed pattern "chains" grouping and aggregation too close
>>>>>> together. I would rather separate both more than less, ie, do into
>> the
>>>>>> opposite direction.
>>>>>>
>>>>>> I am also wondering, if we could so something more "fluent". The
>>> initial
>>>>>> proposal was like:
>>>>>>
>>>>>>>> groupedStream.count()
>>>>>>>>    .withStoreName("name")
>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>    .table()
>>>>>>
>>>>>> The .table() statement in the end was kinda alien.
>>>>>>
>>>>>
>>>>> I agree, but then all of the withXXX methods need to be on KTable
>> which
>>>> is
>>>>> worse in my opinion. You also need something that is going to "build"
>>> the
>>>>> internal processors and add them to the topology.
>>>>>
>>>>>
>>>>>> The current proposal put the count() into the end -- ie, the
>> optional
>>>>>> parameter for count() have to specified on the .grouped() call --
>> this
>>>>>> does not seems to be the best way either.
>>>>>>
>>>>>>
>>>>> I actually prefer this method as you are building a grouped stream
>> that
>>>> you
>>>>> will aggregate. So
>> table.grouped(...).withOptionalStuff().aggregate(..)
>>>> etc
>>>>> seems natural to me.
>>>>>
>>>>>
>>>>>> I did not think this through in detail, but can't we just do the
>>> initial
>>>>>> proposal with the .table() ?
>>>>>>
>>>>>> groupedStream.count().withStoreName("name").mapValues(...)
>>>>>>
>>>>>> Each .withXXX(...) return the current KTable and all the .withXXX()
>>> are
>>>>>> just added to the KTable interface. Or do I miss anything why this
>>> wont'
>>>>>> work or any obvious disadvantage?
>>>>>>
>>>>>>
>>>>>>
>>>>> See above.
>>>>>
>>>>>
>>>>>>
>>>>>> -Matthias
>>>>>>
>>>>>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>>>>>> Thanks everyone. My latest attempt is below. It builds on the
>> fluent
>>>>>>> approach, but i think it is slightly nicer.
>>>>>>> I agree with some of what Eno said about mixing configy stuff in
>> the
>>>> DSL,
>>>>>>> but i think that enabling caching and enabling logging are things
>>> that
>>>>>>> aren't actually config. I'd probably not add withLogConfig(...)
>> (even
>>>>>>> though it is below) as this is actually config and we already have
>> a
>>>> way
>>>>>> of
>>>>>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>>>>>> StateStoreSupplier for disabling caching etc, but as it stands that
>>> is
>>>> a
>>>>>>> bit of a tedious process for someone that just wants to use the
>>> default
>>>>>>> storage engine, but not have caching enabled.
>>>>>>>
>>>>>>> There is also an orthogonal concern that Guozhang alluded to.... If
>>> you
>>>>>>> want to plug in a custom storage engine and you want it to be
>> logged
>>>> etc,
>>>>>>> you would currently need to implement that yourself. Ideally we can
>>>>>> provide
>>>>>>> a way where we will wrap the custom store with logging, metrics,
>>> etc. I
>>>>>>> need to think about where this fits, it is probably more
>> appropriate
>>> on
>>>>>> the
>>>>>>> Stores API.
>>>>>>>
>>>>>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>>>>>> // count with mapped key
>>>>>>> final KTable<Long, Long> count = stream.grouped()
>>>>>>>         .withKeyMapper(keyMapper)
>>>>>>>         .withKeySerde(Serdes.Long())
>>>>>>>         .withValueSerde(Serdes.String())
>>>>>>>         .withQueryableName("my-store")
>>>>>>>         .count();
>>>>>>>
>>>>>>> // windowed count
>>>>>>> final KTable<Windowed<String>, Long> windowedCount =
>> stream.grouped()
>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>         .count();
>>>>>>>
>>>>>>> // windowed reduce
>>>>>>> final Reducer<String> windowedReducer = null;
>>>>>>> final KTable<Windowed<String>, String> windowedReduce =
>>>> stream.grouped()
>>>>>>>         .withQueryableName("my-window-store")
>>>>>>>         .windowed(TimeWindows.of(10L).until(10))
>>>>>>>         .reduce(windowedReducer);
>>>>>>>
>>>>>>> final Aggregator<String, String, Long> aggregator = null;
>>>>>>> final Initializer<Long> init = null;
>>>>>>>
>>>>>>> // aggregate
>>>>>>> final KTable<String, Long> aggregate = stream.grouped()
>>>>>>>         .withQueryableName("my-aggregate-store")
>>>>>>>         .aggregate(aggregator, init, Serdes.Long());
>>>>>>>
>>>>>>> final StateStoreSupplier<KeyValueStore<String, Long>>
>>>> stateStoreSupplier
>>>>>> = null;
>>>>>>>
>>>>>>> // aggregate with custom store
>>>>>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>>>>>         .withStateStoreSupplier(stateStoreSupplier)
>>>>>>>         .aggregate(aggregator, init);
>>>>>>>
>>>>>>> // disable caching
>>>>>>> stream.grouped()
>>>>>>>         .withQueryableName("name")
>>>>>>>         .withCachingEnabled(false)
>>>>>>>         .count();
>>>>>>>
>>>>>>> // disable logging
>>>>>>> stream.grouped()
>>>>>>>         .withQueryableName("q")
>>>>>>>         .withLoggingEnabled(false)
>>>>>>>         .count();
>>>>>>>
>>>>>>> // override log config
>>>>>>> final Reducer<String> reducer = null;
>>>>>>> stream.grouped()
>>>>>>>         .withLogConfig(Collections.singletonMap("segment.size",
>>> "10"))
>>>>>>>         .reduce(reducer);
>>>>>>>
>>>>>>>
>>>>>>> If anyone wants to play around with this you can find the code
>> here:
>>>>>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>>>>>
>>>>>>> Note: It won't actually work as most of the methods just return
>> null.
>>>>>>>
>>>>>>> Thanks,
>>>>>>> Damian
>>>>>>>
>>>>>>>
>>>>>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
>> wrote:
>>>>>>>
>>>>>>>> Thanks Damian. I think both options have pros and cons. And both
>> are
>>>>>> better
>>>>>>>> than overload abuse.
>>>>>>>>
>>>>>>>> The fluent API approach reads better, no mention of builder or
>> build
>>>>>>>> anywhere. The main downside is that the method signatures are a
>>> little
>>>>>> less
>>>>>>>> clear. By reading the method signature, one doesn't necessarily
>>> knows
>>>>>> what
>>>>>>>> it returns. Also, one needs to figure out the special method
>>>> (`table()`
>>>>>> in
>>>>>>>> this case) that gives you what you actually care about (`KTable`
>> in
>>>> this
>>>>>>>> case). Not major issues, but worth mentioning while doing the
>>>>>> comparison.
>>>>>>>>
>>>>>>>> The builder approach avoids the issues mentioned above, but it
>>> doesn't
>>>>>> read
>>>>>>>> as well.
>>>>>>>>
>>>>>>>> Ismael
>>>>>>>>
>>>>>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
>>>
>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> I'd like to get a discussion going around some of the API choices
>>>> we've
>>>>>>>>> made in the DLS. In particular those that relate to stateful
>>>> operations
>>>>>>>>> (though this could expand).
>>>>>>>>> As it stands we lean heavily on overloaded methods in the API,
>> i.e,
>>>>>> there
>>>>>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
>> noisy
>>>> and
>>>>>> i
>>>>>>>>> feel it is only going to get worse as we add more optional
>> params.
>>> In
>>>>>>>>> particular we've had some requests to be able to turn caching
>> off,
>>> or
>>>>>>>>> change log configs,  on a per operator basis (note this can be
>> done
>>>> now
>>>>>>>> if
>>>>>>>>> you pass in a StateStoreSupplier, but this can be a bit
>>> cumbersome).
>>>>>>>>>
>>>>>>>>> So this is a bit of an open question. How can we change the DSL
>>>>>> overloads
>>>>>>>>> so that it flows, is simple to use and understand, and is easily
>>>>>> extended
>>>>>>>>> in the future?
>>>>>>>>>
>>>>>>>>> One option would be to use a fluent API approach for providing
>> the
>>>>>>>> optional
>>>>>>>>> params, so something like this:
>>>>>>>>>
>>>>>>>>> groupedStream.count()
>>>>>>>>>    .withStoreName("name")
>>>>>>>>>    .withCachingEnabled(false)
>>>>>>>>>    .withLoggingEnabled(config)
>>>>>>>>>    .table()
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> Another option would be to provide a Builder to the count method,
>>> so
>>>> it
>>>>>>>>> would look something like this:
>>>>>>>>> groupedStream.count(new
>>>>>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>>>>>
>>>>>>>>> Another option is to say: Hey we don't need this, what are you on
>>>>>> about!
>>>>>>>>>
>>>>>>>>> The above has focussed on state store related overloads, but the
>>> same
>>>>>>>> ideas
>>>>>>>>> could  be applied to joins etc, where we presently have many join
>>>>>> methods
>>>>>>>>> and many overloads.
>>>>>>>>>
>>>>>>>>> Anyway, i look forward to hearing your opinions.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>>
>>
>>
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks Kyle.

On Thu, 29 Jun 2017 at 15:11 Kyle Winkelman <wi...@gmail.com>
wrote:

> Hi Damian,
>
> >>>> When trying to program in the fluent API that has been discussed most
> it
> >>>> feels difficult to know when you will actually get an object you can
> reuse.
> >>>> What if I make one KGroupedStream that I want to reuse, is it legal to
> >>>> reuse it or does this approach expect you to call grouped each time?
>
> >> I'd anticipate that once you have a KGroupedStream you can re-use it as
> you
> >> can today.
>
> You said it yourself in another post that the grouped stream is
> effectively a no-op until a count, reduce, or aggregate. The way I see it
> you wouldn’t be able to reuse anything except KStreams and KTables, because
> most of this fluent api would continue returning this (this being the
> builder object currently being manipulated).

So, if you ever store a reference to anything but KStreams and KTables and
> you use it in two different ways then its possible you make conflicting
> withXXX() calls on the same builder.
>
>
No necessarily true. It could return a new instance of the builder, i.e.,
the builders being immutable. So if you held a reference to the builder it
would always be the same as it was when it was created.


> GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
> GroupedStream<K,V> groupedStreamWithDeclaredSerdes =
> groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);
>
> I’ll admit that this shouldn’t happen but some user is going to do it
> eventually…
> Depending on implementation uses of groupedStreamWithDefaultSerdes would
> most likely be equivalent to the version withDeclaredSerdes. One work
> around would be to always make copies of the config objects you are
> building, but this approach has its own problem because now we have to
> identify which configs are equivalent so we don’t create repeated
> processors.
>
> The point of this long winded example is that we always have to be
> thinking about all of the possible ways it could be misused by a user
> (causing them to see hard to diagnose problems).
>

Exactly! That is the point of the discussion really.


>
> In my attempt at a couple methods with builders I feel that I could
> confidently say the user couldn’t really mess it up.
> > // Count
> > KTable<String, Long> count =
> > kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
> The kGroupedStream is reusable and if they attempted to reuse the Count
> for some reason it would throw an error message saying that a store named
> “my-store” already exists.
>
>
Yes i agree and i think using builders is my preferred pattern.

Cheers,
Damian


> Thanks,
> Kyle
>
> From: Damian Guy
> Sent: Thursday, June 29, 2017 3:59 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>
> Hi Kyle,
>
> Thanks for your input. Really appreciated.
>
> On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
> wrote:
>
> > I like more of a builder pattern even though others have voiced against
> > it. The reason I like it is because it makes it clear to the user that a
> > call to KGroupedStream#count will return a KTable not some intermediate
> > class that I need to undetstand.
> >
>
> Yes, that makes sense.
>
>
> > When trying to program in the fluent API that has been discussed most it
> > feels difficult to know when you will actually get an object you can
> reuse.
> > What if I make one KGroupedStream that I want to reuse, is it legal to
> > reuse it or does this approach expect you to call grouped each time?
>
>
> I'd anticipate that once you have a KGroupedStream you can re-use it as you
> can today.
>
>
> > This question doesn’t pop into my head at all in the builder pattern I
> > assume I can reuse everything.
> > Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan of
> > the grouped.
> >
> > Yes, grouped() was more for demonstration and because groupBy() and
> groupByKey() were taken! So i'd imagine the api would actually want to be
> groupByKey(/** no required args***/).withOptionalArg() and
> groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all depends
> on maintaining backward compatibility.
>
>
> > Unfortunately, the below approach would require atleast 2 (probably 3)
> > overloads (one for returning a KTable and one for returning a KTable with
> > Windowed Key, probably would want to split windowed and sessionwindowed
> for
> > ease of implementation) of each count, reduce, and aggregate.
> > Obviously not exhaustive but enough for you to get the picture. Count,
> > Reduce, and Aggregate supply 3 static methods to initialize the builder:
> > // Count
> > KTable<String, Long> count =
> > groupedStream.count(Count.count().withQueryableStoreName("my-store"));
> >
> > // Windowed Count
> > KTable<Windowed<String>, Long> windowedCount =
> >
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
> >
> > // Session Count
> > KTable<Windowed<String>, Long> sessionCount =
> >
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
> >
> >
> Above and below, i think i'd prefer it to be:
> groupedStream.count(/** non windowed count**/)
> groupedStream.windowed(TimeWindows.of(10L)).count(...)
> groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)
>
>
>
>
> > // Reduce
> > Reducer<Long> reducer;
> > KTable<String, Long> reduce = groupedStream.reduce(reducer,
> > Reduce.reduce().withQueryableStoreName("my-store"));
> >
> > // Aggregate Windowed with Custom Store
> > Initializer<String> initializer;
> > Aggregator<String, Long, String> aggregator;
> > KTable<Windowed<String>, String> aggregate =
> > groupedStream.aggregate(initializer, aggregator,
> >
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
> >
> > // Cogroup SessionWindowed
> > KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
> >         .cogroup(groupedStream2, aggregator2)
> >         .aggregate(initializer, aggregator,
> > Aggregate.sessionWindowed(SessionWindows.with(10L),
> > sessionMerger).withQueryableStoreName("my-store"));
> >
> >
> >
> > public class Count {
> >
> >     public static class Windowed extends Count {
> >         private Windows windows;
> >     }
> >     public static class SessionWindowed extends Count {
> >         private SessionWindows sessionWindows;
> >     }
> >
> >     public static Count count();
> >     public static Windowed windowed(Windows windows);
> >     public static SessionWindowed sessionWindowed(SessionWindows
> > sessionWindows);
> >
> >     // All withXXX(...) methods.
> > }
> >
> > public class KGroupedStream {
> >     public KTable<K, Long> count(Count count);
> >     public KTable<Windowed<K>, Long> count(Count.Windowed count);
> >     public KTable<Windowed<K>, Long> count(Count.SessionWindowed count);
> > …
> > }
> >
> >
> > Thanks,
> > Kyle
> >
> > From: Guozhang Wang
> > Sent: Wednesday, June 28, 2017 7:45 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
> >
> > I played the current proposal a bit with https://github.com/dguy/kafka/
> > tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment>,
> > and here are my observations:
> >
> > 1. Personally I prefer
> >
> >     "stream.group(mapper) / stream.groupByKey()"
> >
> > than
> >
> >     "stream.group().withKeyMapper(mapper) / stream.group()"
> >
> > Since 1) withKeyMapper is not enforced programmatically though it is not
> > "really" optional like others, 2) syntax-wise it reads more natural.
> >
> > I think it is okay to add the APIs in (
> >
> >
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> > )
> > in KGroupedStream.
> >
> >
> > 2. For the "withStateStoreSupplier" API, are the user supposed to pass in
> > the most-inner state store supplier (e.g. then one whose get() return
> > RocksDBStore), or it is supposed to return the most-outer supplier with
> > logging / metrics / etc? I think it would be more useful to only require
> > users pass in the inner state store supplier while specifying caching /
> > logging through other APIs.
> >
> > In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we
> are
> > allowing users to call other APIs like "withQueryableName" multiple time,
> > but only call "withStateStoreSupplier" only once in the end. Why is that?
> >
> >
> > 3. The current DSL seems to be only for aggregations, what about joins?
> >
> >
> > 4. I think it is okay to keep the "withLogConfig": for the
> > StateStoreSupplier it will still be user code specifying the topology so
> I
> > do not see there is a big difference.
> >
> >
> > 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> > windowed state store supplier to enforce typing?
> >
> >
> > Below are minor ones:
> >
> > 6. "withQueryableName": maybe better "withQueryableStateName"?
> >
> > 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
> >
> >
> >
> > Guozhang
> >
> >
> >
> > On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > I see your point about "when to add the processor to the topology".
> That
> > > is indeed an issue. Not sure it we could allow "updates" to the
> > topology...
> > >
> > > I don't see any problem with having all the withXX() in KTable
> interface
> > > -- but this might be subjective.
> > >
> > >
> > > However, I don't understand your argument about putting aggregate()
> > > after the withXX() -- all the calls to withXX() set optional parameters
> > > for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > > indicates that the withXX() belongs to the groupBy(). IMHO, this might
> > > be quite confusion for developers.
> > >
> > >
> > > -Matthias
> > >
> > > On 6/28/17 2:55 AM, Damian Guy wrote:
> > > >> I also think that mixing optional parameters with configs is a bad
> > idea.
> > > >> Have not proposal for this atm but just wanted to mention it. Hope
> to
> > > >> find some time to come up with something.
> > > >>
> > > >>
> > > > Yes, i don't like the mix of config either. But the only real config
> > here
> > > > is the logging config - which we don't really need as it can already
> be
> > > > done via a custom StateStoreSupplier.
> > > >
> > > >
> > > >> What I don't like in the current proposal is the
> > > >> .grouped().withKeyMapper() -- the current solution with
> .groupBy(...)
> > > >> and .groupByKey() seems better. For clarity, we could rename to
> > > >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> > > >> some better names).
> > > >>
> > > >>
> > > > it could be groupByKey(), groupBy() or something different bt
> > > >
> > > >
> > > >
> > > >> The proposed pattern "chains" grouping and aggregation too close
> > > >> together. I would rather separate both more than less, ie, do into
> the
> > > >> opposite direction.
> > > >>
> > > >> I am also wondering, if we could so something more "fluent". The
> > initial
> > > >> proposal was like:
> > > >>
> > > >>>> groupedStream.count()
> > > >>>>    .withStoreName("name")
> > > >>>>    .withCachingEnabled(false)
> > > >>>>    .withLoggingEnabled(config)
> > > >>>>    .table()
> > > >>
> > > >> The .table() statement in the end was kinda alien.
> > > >>
> > > >
> > > > I agree, but then all of the withXXX methods need to be on KTable
> which
> > > is
> > > > worse in my opinion. You also need something that is going to "build"
> > the
> > > > internal processors and add them to the topology.
> > > >
> > > >
> > > >> The current proposal put the count() into the end -- ie, the
> optional
> > > >> parameter for count() have to specified on the .grouped() call --
> this
> > > >> does not seems to be the best way either.
> > > >>
> > > >>
> > > > I actually prefer this method as you are building a grouped stream
> that
> > > you
> > > > will aggregate. So
> table.grouped(...).withOptionalStuff().aggregate(..)
> > > etc
> > > > seems natural to me.
> > > >
> > > >
> > > >> I did not think this through in detail, but can't we just do the
> > initial
> > > >> proposal with the .table() ?
> > > >>
> > > >> groupedStream.count().withStoreName("name").mapValues(...)
> > > >>
> > > >> Each .withXXX(...) return the current KTable and all the .withXXX()
> > are
> > > >> just added to the KTable interface. Or do I miss anything why this
> > wont'
> > > >> work or any obvious disadvantage?
> > > >>
> > > >>
> > > >>
> > > > See above.
> > > >
> > > >
> > > >>
> > > >> -Matthias
> > > >>
> > > >> On 6/22/17 4:06 AM, Damian Guy wrote:
> > > >>> Thanks everyone. My latest attempt is below. It builds on the
> fluent
> > > >>> approach, but i think it is slightly nicer.
> > > >>> I agree with some of what Eno said about mixing configy stuff in
> the
> > > DSL,
> > > >>> but i think that enabling caching and enabling logging are things
> > that
> > > >>> aren't actually config. I'd probably not add withLogConfig(...)
> (even
> > > >>> though it is below) as this is actually config and we already have
> a
> > > way
> > > >> of
> > > >>> doing that, via the StateStoreSupplier. Arguably we could use the
> > > >>> StateStoreSupplier for disabling caching etc, but as it stands that
> > is
> > > a
> > > >>> bit of a tedious process for someone that just wants to use the
> > default
> > > >>> storage engine, but not have caching enabled.
> > > >>>
> > > >>> There is also an orthogonal concern that Guozhang alluded to.... If
> > you
> > > >>> want to plug in a custom storage engine and you want it to be
> logged
> > > etc,
> > > >>> you would currently need to implement that yourself. Ideally we can
> > > >> provide
> > > >>> a way where we will wrap the custom store with logging, metrics,
> > etc. I
> > > >>> need to think about where this fits, it is probably more
> appropriate
> > on
> > > >> the
> > > >>> Stores API.
> > > >>>
> > > >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > > >>> // count with mapped key
> > > >>> final KTable<Long, Long> count = stream.grouped()
> > > >>>         .withKeyMapper(keyMapper)
> > > >>>         .withKeySerde(Serdes.Long())
> > > >>>         .withValueSerde(Serdes.String())
> > > >>>         .withQueryableName("my-store")
> > > >>>         .count();
> > > >>>
> > > >>> // windowed count
> > > >>> final KTable<Windowed<String>, Long> windowedCount =
> stream.grouped()
> > > >>>         .withQueryableName("my-window-store")
> > > >>>         .windowed(TimeWindows.of(10L).until(10))
> > > >>>         .count();
> > > >>>
> > > >>> // windowed reduce
> > > >>> final Reducer<String> windowedReducer = null;
> > > >>> final KTable<Windowed<String>, String> windowedReduce =
> > > stream.grouped()
> > > >>>         .withQueryableName("my-window-store")
> > > >>>         .windowed(TimeWindows.of(10L).until(10))
> > > >>>         .reduce(windowedReducer);
> > > >>>
> > > >>> final Aggregator<String, String, Long> aggregator = null;
> > > >>> final Initializer<Long> init = null;
> > > >>>
> > > >>> // aggregate
> > > >>> final KTable<String, Long> aggregate = stream.grouped()
> > > >>>         .withQueryableName("my-aggregate-store")
> > > >>>         .aggregate(aggregator, init, Serdes.Long());
> > > >>>
> > > >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > > stateStoreSupplier
> > > >> = null;
> > > >>>
> > > >>> // aggregate with custom store
> > > >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> > > >>>         .withStateStoreSupplier(stateStoreSupplier)
> > > >>>         .aggregate(aggregator, init);
> > > >>>
> > > >>> // disable caching
> > > >>> stream.grouped()
> > > >>>         .withQueryableName("name")
> > > >>>         .withCachingEnabled(false)
> > > >>>         .count();
> > > >>>
> > > >>> // disable logging
> > > >>> stream.grouped()
> > > >>>         .withQueryableName("q")
> > > >>>         .withLoggingEnabled(false)
> > > >>>         .count();
> > > >>>
> > > >>> // override log config
> > > >>> final Reducer<String> reducer = null;
> > > >>> stream.grouped()
> > > >>>         .withLogConfig(Collections.singletonMap("segment.size",
> > "10"))
> > > >>>         .reduce(reducer);
> > > >>>
> > > >>>
> > > >>> If anyone wants to play around with this you can find the code
> here:
> > > >>> https://github.com/dguy/kafka/tree/dsl-experiment
> > > >>>
> > > >>> Note: It won't actually work as most of the methods just return
> null.
> > > >>>
> > > >>> Thanks,
> > > >>> Damian
> > > >>>
> > > >>>
> > > >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk>
> wrote:
> > > >>>
> > > >>>> Thanks Damian. I think both options have pros and cons. And both
> are
> > > >> better
> > > >>>> than overload abuse.
> > > >>>>
> > > >>>> The fluent API approach reads better, no mention of builder or
> build
> > > >>>> anywhere. The main downside is that the method signatures are a
> > little
> > > >> less
> > > >>>> clear. By reading the method signature, one doesn't necessarily
> > knows
> > > >> what
> > > >>>> it returns. Also, one needs to figure out the special method
> > > (`table()`
> > > >> in
> > > >>>> this case) that gives you what you actually care about (`KTable`
> in
> > > this
> > > >>>> case). Not major issues, but worth mentioning while doing the
> > > >> comparison.
> > > >>>>
> > > >>>> The builder approach avoids the issues mentioned above, but it
> > doesn't
> > > >> read
> > > >>>> as well.
> > > >>>>
> > > >>>> Ismael
> > > >>>>
> > > >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <damian.guy@gmail.com
> >
> > > >> wrote:
> > > >>>>
> > > >>>>> Hi,
> > > >>>>>
> > > >>>>> I'd like to get a discussion going around some of the API choices
> > > we've
> > > >>>>> made in the DLS. In particular those that relate to stateful
> > > operations
> > > >>>>> (though this could expand).
> > > >>>>> As it stands we lean heavily on overloaded methods in the API,
> i.e,
> > > >> there
> > > >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming
> noisy
> > > and
> > > >> i
> > > >>>>> feel it is only going to get worse as we add more optional
> params.
> > In
> > > >>>>> particular we've had some requests to be able to turn caching
> off,
> > or
> > > >>>>> change log configs,  on a per operator basis (note this can be
> done
> > > now
> > > >>>> if
> > > >>>>> you pass in a StateStoreSupplier, but this can be a bit
> > cumbersome).
> > > >>>>>
> > > >>>>> So this is a bit of an open question. How can we change the DSL
> > > >> overloads
> > > >>>>> so that it flows, is simple to use and understand, and is easily
> > > >> extended
> > > >>>>> in the future?
> > > >>>>>
> > > >>>>> One option would be to use a fluent API approach for providing
> the
> > > >>>> optional
> > > >>>>> params, so something like this:
> > > >>>>>
> > > >>>>> groupedStream.count()
> > > >>>>>    .withStoreName("name")
> > > >>>>>    .withCachingEnabled(false)
> > > >>>>>    .withLoggingEnabled(config)
> > > >>>>>    .table()
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> Another option would be to provide a Builder to the count method,
> > so
> > > it
> > > >>>>> would look something like this:
> > > >>>>> groupedStream.count(new
> > > >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > > >>>>>
> > > >>>>> Another option is to say: Hey we don't need this, what are you on
> > > >> about!
> > > >>>>>
> > > >>>>> The above has focussed on state store related overloads, but the
> > same
> > > >>>> ideas
> > > >>>>> could  be applied to joins etc, where we presently have many join
> > > >> methods
> > > >>>>> and many overloads.
> > > >>>>>
> > > >>>>> Anyway, i look forward to hearing your opinions.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Damian
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > > >>
> > > >
> > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
> >
>
>

RE: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Kyle Winkelman <wi...@gmail.com>.
Hi Damian,

>>>> When trying to program in the fluent API that has been discussed most it
>>>> feels difficult to know when you will actually get an object you can reuse.
>>>> What if I make one KGroupedStream that I want to reuse, is it legal to
>>>> reuse it or does this approach expect you to call grouped each time?

>> I'd anticipate that once you have a KGroupedStream you can re-use it as you
>> can today.

You said it yourself in another post that the grouped stream is effectively a no-op until a count, reduce, or aggregate. The way I see it you wouldn’t be able to reuse anything except KStreams and KTables, because most of this fluent api would continue returning this (this being the builder object currently being manipulated). So, if you ever store a reference to anything but KStreams and KTables and you use it in two different ways then its possible you make conflicting withXXX() calls on the same builder.

GroupedStream<K,V> groupedStreamWithDefaultSerdes = kStream.grouped();
GroupedStream<K,V> groupedStreamWithDeclaredSerdes = groupedStreamsWithDefaultSerdes.withKeySerde(…).withValueSerde(…);

I’ll admit that this shouldn’t happen but some user is going to do it eventually…
Depending on implementation uses of groupedStreamWithDefaultSerdes would most likely be equivalent to the version withDeclaredSerdes. One work around would be to always make copies of the config objects you are building, but this approach has its own problem because now we have to identify which configs are equivalent so we don’t create repeated processors.

The point of this long winded example is that we always have to be thinking about all of the possible ways it could be misused by a user (causing them to see hard to diagnose problems).

In my attempt at a couple methods with builders I feel that I could confidently say the user couldn’t really mess it up.
> // Count
> KTable<String, Long> count =
> kGroupedStream.count(Count.count().withQueryableStoreName("my-store"));
The kGroupedStream is reusable and if they attempted to reuse the Count for some reason it would throw an error message saying that a store named “my-store” already exists.

Thanks,
Kyle

From: Damian Guy
Sent: Thursday, June 29, 2017 3:59 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring

Hi Kyle,

Thanks for your input. Really appreciated.

On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
wrote:

> I like more of a builder pattern even though others have voiced against
> it. The reason I like it is because it makes it clear to the user that a
> call to KGroupedStream#count will return a KTable not some intermediate
> class that I need to undetstand.
>

Yes, that makes sense.


> When trying to program in the fluent API that has been discussed most it
> feels difficult to know when you will actually get an object you can reuse.
> What if I make one KGroupedStream that I want to reuse, is it legal to
> reuse it or does this approach expect you to call grouped each time?


I'd anticipate that once you have a KGroupedStream you can re-use it as you
can today.


> This question doesn’t pop into my head at all in the builder pattern I
> assume I can reuse everything.
> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan of
> the grouped.
>
> Yes, grouped() was more for demonstration and because groupBy() and
groupByKey() were taken! So i'd imagine the api would actually want to be
groupByKey(/** no required args***/).withOptionalArg() and
groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all depends
on maintaining backward compatibility.


> Unfortunately, the below approach would require atleast 2 (probably 3)
> overloads (one for returning a KTable and one for returning a KTable with
> Windowed Key, probably would want to split windowed and sessionwindowed for
> ease of implementation) of each count, reduce, and aggregate.
> Obviously not exhaustive but enough for you to get the picture. Count,
> Reduce, and Aggregate supply 3 static methods to initialize the builder:
> // Count
> KTable<String, Long> count =
> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
> // Windowed Count
> KTable<Windowed<String>, Long> windowedCount =
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>
> // Session Count
> KTable<Windowed<String>, Long> sessionCount =
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>
>
Above and below, i think i'd prefer it to be:
groupedStream.count(/** non windowed count**/)
groupedStream.windowed(TimeWindows.of(10L)).count(...)
groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)




> // Reduce
> Reducer<Long> reducer;
> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> Reduce.reduce().withQueryableStoreName("my-store"));
>
> // Aggregate Windowed with Custom Store
> Initializer<String> initializer;
> Aggregator<String, Long, String> aggregator;
> KTable<Windowed<String>, String> aggregate =
> groupedStream.aggregate(initializer, aggregator,
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>
> // Cogroup SessionWindowed
> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>         .cogroup(groupedStream2, aggregator2)
>         .aggregate(initializer, aggregator,
> Aggregate.sessionWindowed(SessionWindows.with(10L),
> sessionMerger).withQueryableStoreName("my-store"));
>
>
>
> public class Count {
>
>     public static class Windowed extends Count {
>         private Windows windows;
>     }
>     public static class SessionWindowed extends Count {
>         private SessionWindows sessionWindows;
>     }
>
>     public static Count count();
>     public static Windowed windowed(Windows windows);
>     public static SessionWindowed sessionWindowed(SessionWindows
> sessionWindows);
>
>     // All withXXX(...) methods.
> }
>
> public class KGroupedStream {
>     public KTable<K, Long> count(Count count);
>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed count);
> …
> }
>
>
> Thanks,
> Kyle
>
> From: Guozhang Wang
> Sent: Wednesday, June 28, 2017 7:45 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>
> I played the current proposal a bit with https://github.com/dguy/kafka/
> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment>,
> and here are my observations:
>
> 1. Personally I prefer
>
>     "stream.group(mapper) / stream.groupByKey()"
>
> than
>
>     "stream.group().withKeyMapper(mapper) / stream.group()"
>
> Since 1) withKeyMapper is not enforced programmatically though it is not
> "really" optional like others, 2) syntax-wise it reads more natural.
>
> I think it is okay to add the APIs in (
>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> )
> in KGroupedStream.
>
>
> 2. For the "withStateStoreSupplier" API, are the user supposed to pass in
> the most-inner state store supplier (e.g. then one whose get() return
> RocksDBStore), or it is supposed to return the most-outer supplier with
> logging / metrics / etc? I think it would be more useful to only require
> users pass in the inner state store supplier while specifying caching /
> logging through other APIs.
>
> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we are
> allowing users to call other APIs like "withQueryableName" multiple time,
> but only call "withStateStoreSupplier" only once in the end. Why is that?
>
>
> 3. The current DSL seems to be only for aggregations, what about joins?
>
>
> 4. I think it is okay to keep the "withLogConfig": for the
> StateStoreSupplier it will still be user code specifying the topology so I
> do not see there is a big difference.
>
>
> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> windowed state store supplier to enforce typing?
>
>
> Below are minor ones:
>
> 6. "withQueryableName": maybe better "withQueryableStateName"?
>
> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>
>
>
> Guozhang
>
>
>
> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > I see your point about "when to add the processor to the topology". That
> > is indeed an issue. Not sure it we could allow "updates" to the
> topology...
> >
> > I don't see any problem with having all the withXX() in KTable interface
> > -- but this might be subjective.
> >
> >
> > However, I don't understand your argument about putting aggregate()
> > after the withXX() -- all the calls to withXX() set optional parameters
> > for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > indicates that the withXX() belongs to the groupBy(). IMHO, this might
> > be quite confusion for developers.
> >
> >
> > -Matthias
> >
> > On 6/28/17 2:55 AM, Damian Guy wrote:
> > >> I also think that mixing optional parameters with configs is a bad
> idea.
> > >> Have not proposal for this atm but just wanted to mention it. Hope to
> > >> find some time to come up with something.
> > >>
> > >>
> > > Yes, i don't like the mix of config either. But the only real config
> here
> > > is the logging config - which we don't really need as it can already be
> > > done via a custom StateStoreSupplier.
> > >
> > >
> > >> What I don't like in the current proposal is the
> > >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> > >> and .groupByKey() seems better. For clarity, we could rename to
> > >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> > >> some better names).
> > >>
> > >>
> > > it could be groupByKey(), groupBy() or something different bt
> > >
> > >
> > >
> > >> The proposed pattern "chains" grouping and aggregation too close
> > >> together. I would rather separate both more than less, ie, do into the
> > >> opposite direction.
> > >>
> > >> I am also wondering, if we could so something more "fluent". The
> initial
> > >> proposal was like:
> > >>
> > >>>> groupedStream.count()
> > >>>>    .withStoreName("name")
> > >>>>    .withCachingEnabled(false)
> > >>>>    .withLoggingEnabled(config)
> > >>>>    .table()
> > >>
> > >> The .table() statement in the end was kinda alien.
> > >>
> > >
> > > I agree, but then all of the withXXX methods need to be on KTable which
> > is
> > > worse in my opinion. You also need something that is going to "build"
> the
> > > internal processors and add them to the topology.
> > >
> > >
> > >> The current proposal put the count() into the end -- ie, the optional
> > >> parameter for count() have to specified on the .grouped() call -- this
> > >> does not seems to be the best way either.
> > >>
> > >>
> > > I actually prefer this method as you are building a grouped stream that
> > you
> > > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> > etc
> > > seems natural to me.
> > >
> > >
> > >> I did not think this through in detail, but can't we just do the
> initial
> > >> proposal with the .table() ?
> > >>
> > >> groupedStream.count().withStoreName("name").mapValues(...)
> > >>
> > >> Each .withXXX(...) return the current KTable and all the .withXXX()
> are
> > >> just added to the KTable interface. Or do I miss anything why this
> wont'
> > >> work or any obvious disadvantage?
> > >>
> > >>
> > >>
> > > See above.
> > >
> > >
> > >>
> > >> -Matthias
> > >>
> > >> On 6/22/17 4:06 AM, Damian Guy wrote:
> > >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> > >>> approach, but i think it is slightly nicer.
> > >>> I agree with some of what Eno said about mixing configy stuff in the
> > DSL,
> > >>> but i think that enabling caching and enabling logging are things
> that
> > >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> > >>> though it is below) as this is actually config and we already have a
> > way
> > >> of
> > >>> doing that, via the StateStoreSupplier. Arguably we could use the
> > >>> StateStoreSupplier for disabling caching etc, but as it stands that
> is
> > a
> > >>> bit of a tedious process for someone that just wants to use the
> default
> > >>> storage engine, but not have caching enabled.
> > >>>
> > >>> There is also an orthogonal concern that Guozhang alluded to.... If
> you
> > >>> want to plug in a custom storage engine and you want it to be logged
> > etc,
> > >>> you would currently need to implement that yourself. Ideally we can
> > >> provide
> > >>> a way where we will wrap the custom store with logging, metrics,
> etc. I
> > >>> need to think about where this fits, it is probably more appropriate
> on
> > >> the
> > >>> Stores API.
> > >>>
> > >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > >>> // count with mapped key
> > >>> final KTable<Long, Long> count = stream.grouped()
> > >>>         .withKeyMapper(keyMapper)
> > >>>         .withKeySerde(Serdes.Long())
> > >>>         .withValueSerde(Serdes.String())
> > >>>         .withQueryableName("my-store")
> > >>>         .count();
> > >>>
> > >>> // windowed count
> > >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .count();
> > >>>
> > >>> // windowed reduce
> > >>> final Reducer<String> windowedReducer = null;
> > >>> final KTable<Windowed<String>, String> windowedReduce =
> > stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .reduce(windowedReducer);
> > >>>
> > >>> final Aggregator<String, String, Long> aggregator = null;
> > >>> final Initializer<Long> init = null;
> > >>>
> > >>> // aggregate
> > >>> final KTable<String, Long> aggregate = stream.grouped()
> > >>>         .withQueryableName("my-aggregate-store")
> > >>>         .aggregate(aggregator, init, Serdes.Long());
> > >>>
> > >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > stateStoreSupplier
> > >> = null;
> > >>>
> > >>> // aggregate with custom store
> > >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> > >>>         .withStateStoreSupplier(stateStoreSupplier)
> > >>>         .aggregate(aggregator, init);
> > >>>
> > >>> // disable caching
> > >>> stream.grouped()
> > >>>         .withQueryableName("name")
> > >>>         .withCachingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // disable logging
> > >>> stream.grouped()
> > >>>         .withQueryableName("q")
> > >>>         .withLoggingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // override log config
> > >>> final Reducer<String> reducer = null;
> > >>> stream.grouped()
> > >>>         .withLogConfig(Collections.singletonMap("segment.size",
> "10"))
> > >>>         .reduce(reducer);
> > >>>
> > >>>
> > >>> If anyone wants to play around with this you can find the code here:
> > >>> https://github.com/dguy/kafka/tree/dsl-experiment
> > >>>
> > >>> Note: It won't actually work as most of the methods just return null.
> > >>>
> > >>> Thanks,
> > >>> Damian
> > >>>
> > >>>
> > >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> > >>>
> > >>>> Thanks Damian. I think both options have pros and cons. And both are
> > >> better
> > >>>> than overload abuse.
> > >>>>
> > >>>> The fluent API approach reads better, no mention of builder or build
> > >>>> anywhere. The main downside is that the method signatures are a
> little
> > >> less
> > >>>> clear. By reading the method signature, one doesn't necessarily
> knows
> > >> what
> > >>>> it returns. Also, one needs to figure out the special method
> > (`table()`
> > >> in
> > >>>> this case) that gives you what you actually care about (`KTable` in
> > this
> > >>>> case). Not major issues, but worth mentioning while doing the
> > >> comparison.
> > >>>>
> > >>>> The builder approach avoids the issues mentioned above, but it
> doesn't
> > >> read
> > >>>> as well.
> > >>>>
> > >>>> Ismael
> > >>>>
> > >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> > >> wrote:
> > >>>>
> > >>>>> Hi,
> > >>>>>
> > >>>>> I'd like to get a discussion going around some of the API choices
> > we've
> > >>>>> made in the DLS. In particular those that relate to stateful
> > operations
> > >>>>> (though this could expand).
> > >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> > >> there
> > >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> > and
> > >> i
> > >>>>> feel it is only going to get worse as we add more optional params.
> In
> > >>>>> particular we've had some requests to be able to turn caching off,
> or
> > >>>>> change log configs,  on a per operator basis (note this can be done
> > now
> > >>>> if
> > >>>>> you pass in a StateStoreSupplier, but this can be a bit
> cumbersome).
> > >>>>>
> > >>>>> So this is a bit of an open question. How can we change the DSL
> > >> overloads
> > >>>>> so that it flows, is simple to use and understand, and is easily
> > >> extended
> > >>>>> in the future?
> > >>>>>
> > >>>>> One option would be to use a fluent API approach for providing the
> > >>>> optional
> > >>>>> params, so something like this:
> > >>>>>
> > >>>>> groupedStream.count()
> > >>>>>    .withStoreName("name")
> > >>>>>    .withCachingEnabled(false)
> > >>>>>    .withLoggingEnabled(config)
> > >>>>>    .table()
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Another option would be to provide a Builder to the count method,
> so
> > it
> > >>>>> would look something like this:
> > >>>>> groupedStream.count(new
> > >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>>>>
> > >>>>> Another option is to say: Hey we don't need this, what are you on
> > >> about!
> > >>>>>
> > >>>>> The above has focussed on state store related overloads, but the
> same
> > >>>> ideas
> > >>>>> could  be applied to joins etc, where we presently have many join
> > >> methods
> > >>>>> and many overloads.
> > >>>>>
> > >>>>> Anyway, i look forward to hearing your opinions.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Damian
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >>
> > >
> >
> >
>
>
> --
> -- Guozhang
>
>


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Hi Kyle,

Thanks for your input. Really appreciated.

On Thu, 29 Jun 2017 at 06:09 Kyle Winkelman <wi...@gmail.com>
wrote:

> I like more of a builder pattern even though others have voiced against
> it. The reason I like it is because it makes it clear to the user that a
> call to KGroupedStream#count will return a KTable not some intermediate
> class that I need to undetstand.
>

Yes, that makes sense.


> When trying to program in the fluent API that has been discussed most it
> feels difficult to know when you will actually get an object you can reuse.
> What if I make one KGroupedStream that I want to reuse, is it legal to
> reuse it or does this approach expect you to call grouped each time?


I'd anticipate that once you have a KGroupedStream you can re-use it as you
can today.


> This question doesn’t pop into my head at all in the builder pattern I
> assume I can reuse everything.
> Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan of
> the grouped.
>
> Yes, grouped() was more for demonstration and because groupBy() and
groupByKey() were taken! So i'd imagine the api would actually want to be
groupByKey(/** no required args***/).withOptionalArg() and
groupBy(KeyValueMapper m).withOpitionalArg(...)  of course this all depends
on maintaining backward compatibility.


> Unfortunately, the below approach would require atleast 2 (probably 3)
> overloads (one for returning a KTable and one for returning a KTable with
> Windowed Key, probably would want to split windowed and sessionwindowed for
> ease of implementation) of each count, reduce, and aggregate.
> Obviously not exhaustive but enough for you to get the picture. Count,
> Reduce, and Aggregate supply 3 static methods to initialize the builder:
> // Count
> KTable<String, Long> count =
> groupedStream.count(Count.count().withQueryableStoreName("my-store"));
>
> // Windowed Count
> KTable<Windowed<String>, Long> windowedCount =
> groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));
>
> // Session Count
> KTable<Windowed<String>, Long> sessionCount =
> groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));
>
>
Above and below, i think i'd prefer it to be:
groupedStream.count(/** non windowed count**/)
groupedStream.windowed(TimeWindows.of(10L)).count(...)
groupedStream.sessionWindowed(SessionWindows.of(10L)).count(...)




> // Reduce
> Reducer<Long> reducer;
> KTable<String, Long> reduce = groupedStream.reduce(reducer,
> Reduce.reduce().withQueryableStoreName("my-store"));
>
> // Aggregate Windowed with Custom Store
> Initializer<String> initializer;
> Aggregator<String, Long, String> aggregator;
> KTable<Windowed<String>, String> aggregate =
> groupedStream.aggregate(initializer, aggregator,
> Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));
>
> // Cogroup SessionWindowed
> KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
>         .cogroup(groupedStream2, aggregator2)
>         .aggregate(initializer, aggregator,
> Aggregate.sessionWindowed(SessionWindows.with(10L),
> sessionMerger).withQueryableStoreName("my-store"));
>
>
>
> public class Count {
>
>     public static class Windowed extends Count {
>         private Windows windows;
>     }
>     public static class SessionWindowed extends Count {
>         private SessionWindows sessionWindows;
>     }
>
>     public static Count count();
>     public static Windowed windowed(Windows windows);
>     public static SessionWindowed sessionWindowed(SessionWindows
> sessionWindows);
>
>     // All withXXX(...) methods.
> }
>
> public class KGroupedStream {
>     public KTable<K, Long> count(Count count);
>     public KTable<Windowed<K>, Long> count(Count.Windowed count);
>     public KTable<Windowed<K>, Long> count(Count.SessionWindowed count);
> …
> }
>
>
> Thanks,
> Kyle
>
> From: Guozhang Wang
> Sent: Wednesday, June 28, 2017 7:45 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring
>
> I played the current proposal a bit with https://github.com/dguy/kafka/
> tree/dsl-experiment <https://github.com/dguy/kafka/tree/dsl-experiment>,
> and here are my observations:
>
> 1. Personally I prefer
>
>     "stream.group(mapper) / stream.groupByKey()"
>
> than
>
>     "stream.group().withKeyMapper(mapper) / stream.group()"
>
> Since 1) withKeyMapper is not enforced programmatically though it is not
> "really" optional like others, 2) syntax-wise it reads more natural.
>
> I think it is okay to add the APIs in (
>
> https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java
> )
> in KGroupedStream.
>
>
> 2. For the "withStateStoreSupplier" API, are the user supposed to pass in
> the most-inner state store supplier (e.g. then one whose get() return
> RocksDBStore), or it is supposed to return the most-outer supplier with
> logging / metrics / etc? I think it would be more useful to only require
> users pass in the inner state store supplier while specifying caching /
> logging through other APIs.
>
> In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we are
> allowing users to call other APIs like "withQueryableName" multiple time,
> but only call "withStateStoreSupplier" only once in the end. Why is that?
>
>
> 3. The current DSL seems to be only for aggregations, what about joins?
>
>
> 4. I think it is okay to keep the "withLogConfig": for the
> StateStoreSupplier it will still be user code specifying the topology so I
> do not see there is a big difference.
>
>
> 5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
> windowed state store supplier to enforce typing?
>
>
> Below are minor ones:
>
> 6. "withQueryableName": maybe better "withQueryableStateName"?
>
> 7. "withLogConfig": maybe better "withLoggingTopicConfig()"?
>
>
>
> Guozhang
>
>
>
> On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > I see your point about "when to add the processor to the topology". That
> > is indeed an issue. Not sure it we could allow "updates" to the
> topology...
> >
> > I don't see any problem with having all the withXX() in KTable interface
> > -- but this might be subjective.
> >
> >
> > However, I don't understand your argument about putting aggregate()
> > after the withXX() -- all the calls to withXX() set optional parameters
> > for aggregate() and not for groupBy() -- but a groupBy().withXX()
> > indicates that the withXX() belongs to the groupBy(). IMHO, this might
> > be quite confusion for developers.
> >
> >
> > -Matthias
> >
> > On 6/28/17 2:55 AM, Damian Guy wrote:
> > >> I also think that mixing optional parameters with configs is a bad
> idea.
> > >> Have not proposal for this atm but just wanted to mention it. Hope to
> > >> find some time to come up with something.
> > >>
> > >>
> > > Yes, i don't like the mix of config either. But the only real config
> here
> > > is the logging config - which we don't really need as it can already be
> > > done via a custom StateStoreSupplier.
> > >
> > >
> > >> What I don't like in the current proposal is the
> > >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> > >> and .groupByKey() seems better. For clarity, we could rename to
> > >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> > >> some better names).
> > >>
> > >>
> > > it could be groupByKey(), groupBy() or something different bt
> > >
> > >
> > >
> > >> The proposed pattern "chains" grouping and aggregation too close
> > >> together. I would rather separate both more than less, ie, do into the
> > >> opposite direction.
> > >>
> > >> I am also wondering, if we could so something more "fluent". The
> initial
> > >> proposal was like:
> > >>
> > >>>> groupedStream.count()
> > >>>>    .withStoreName("name")
> > >>>>    .withCachingEnabled(false)
> > >>>>    .withLoggingEnabled(config)
> > >>>>    .table()
> > >>
> > >> The .table() statement in the end was kinda alien.
> > >>
> > >
> > > I agree, but then all of the withXXX methods need to be on KTable which
> > is
> > > worse in my opinion. You also need something that is going to "build"
> the
> > > internal processors and add them to the topology.
> > >
> > >
> > >> The current proposal put the count() into the end -- ie, the optional
> > >> parameter for count() have to specified on the .grouped() call -- this
> > >> does not seems to be the best way either.
> > >>
> > >>
> > > I actually prefer this method as you are building a grouped stream that
> > you
> > > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> > etc
> > > seems natural to me.
> > >
> > >
> > >> I did not think this through in detail, but can't we just do the
> initial
> > >> proposal with the .table() ?
> > >>
> > >> groupedStream.count().withStoreName("name").mapValues(...)
> > >>
> > >> Each .withXXX(...) return the current KTable and all the .withXXX()
> are
> > >> just added to the KTable interface. Or do I miss anything why this
> wont'
> > >> work or any obvious disadvantage?
> > >>
> > >>
> > >>
> > > See above.
> > >
> > >
> > >>
> > >> -Matthias
> > >>
> > >> On 6/22/17 4:06 AM, Damian Guy wrote:
> > >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> > >>> approach, but i think it is slightly nicer.
> > >>> I agree with some of what Eno said about mixing configy stuff in the
> > DSL,
> > >>> but i think that enabling caching and enabling logging are things
> that
> > >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> > >>> though it is below) as this is actually config and we already have a
> > way
> > >> of
> > >>> doing that, via the StateStoreSupplier. Arguably we could use the
> > >>> StateStoreSupplier for disabling caching etc, but as it stands that
> is
> > a
> > >>> bit of a tedious process for someone that just wants to use the
> default
> > >>> storage engine, but not have caching enabled.
> > >>>
> > >>> There is also an orthogonal concern that Guozhang alluded to.... If
> you
> > >>> want to plug in a custom storage engine and you want it to be logged
> > etc,
> > >>> you would currently need to implement that yourself. Ideally we can
> > >> provide
> > >>> a way where we will wrap the custom store with logging, metrics,
> etc. I
> > >>> need to think about where this fits, it is probably more appropriate
> on
> > >> the
> > >>> Stores API.
> > >>>
> > >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> > >>> // count with mapped key
> > >>> final KTable<Long, Long> count = stream.grouped()
> > >>>         .withKeyMapper(keyMapper)
> > >>>         .withKeySerde(Serdes.Long())
> > >>>         .withValueSerde(Serdes.String())
> > >>>         .withQueryableName("my-store")
> > >>>         .count();
> > >>>
> > >>> // windowed count
> > >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .count();
> > >>>
> > >>> // windowed reduce
> > >>> final Reducer<String> windowedReducer = null;
> > >>> final KTable<Windowed<String>, String> windowedReduce =
> > stream.grouped()
> > >>>         .withQueryableName("my-window-store")
> > >>>         .windowed(TimeWindows.of(10L).until(10))
> > >>>         .reduce(windowedReducer);
> > >>>
> > >>> final Aggregator<String, String, Long> aggregator = null;
> > >>> final Initializer<Long> init = null;
> > >>>
> > >>> // aggregate
> > >>> final KTable<String, Long> aggregate = stream.grouped()
> > >>>         .withQueryableName("my-aggregate-store")
> > >>>         .aggregate(aggregator, init, Serdes.Long());
> > >>>
> > >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> > stateStoreSupplier
> > >> = null;
> > >>>
> > >>> // aggregate with custom store
> > >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> > >>>         .withStateStoreSupplier(stateStoreSupplier)
> > >>>         .aggregate(aggregator, init);
> > >>>
> > >>> // disable caching
> > >>> stream.grouped()
> > >>>         .withQueryableName("name")
> > >>>         .withCachingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // disable logging
> > >>> stream.grouped()
> > >>>         .withQueryableName("q")
> > >>>         .withLoggingEnabled(false)
> > >>>         .count();
> > >>>
> > >>> // override log config
> > >>> final Reducer<String> reducer = null;
> > >>> stream.grouped()
> > >>>         .withLogConfig(Collections.singletonMap("segment.size",
> "10"))
> > >>>         .reduce(reducer);
> > >>>
> > >>>
> > >>> If anyone wants to play around with this you can find the code here:
> > >>> https://github.com/dguy/kafka/tree/dsl-experiment
> > >>>
> > >>> Note: It won't actually work as most of the methods just return null.
> > >>>
> > >>> Thanks,
> > >>> Damian
> > >>>
> > >>>
> > >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> > >>>
> > >>>> Thanks Damian. I think both options have pros and cons. And both are
> > >> better
> > >>>> than overload abuse.
> > >>>>
> > >>>> The fluent API approach reads better, no mention of builder or build
> > >>>> anywhere. The main downside is that the method signatures are a
> little
> > >> less
> > >>>> clear. By reading the method signature, one doesn't necessarily
> knows
> > >> what
> > >>>> it returns. Also, one needs to figure out the special method
> > (`table()`
> > >> in
> > >>>> this case) that gives you what you actually care about (`KTable` in
> > this
> > >>>> case). Not major issues, but worth mentioning while doing the
> > >> comparison.
> > >>>>
> > >>>> The builder approach avoids the issues mentioned above, but it
> doesn't
> > >> read
> > >>>> as well.
> > >>>>
> > >>>> Ismael
> > >>>>
> > >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> > >> wrote:
> > >>>>
> > >>>>> Hi,
> > >>>>>
> > >>>>> I'd like to get a discussion going around some of the API choices
> > we've
> > >>>>> made in the DLS. In particular those that relate to stateful
> > operations
> > >>>>> (though this could expand).
> > >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> > >> there
> > >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> > and
> > >> i
> > >>>>> feel it is only going to get worse as we add more optional params.
> In
> > >>>>> particular we've had some requests to be able to turn caching off,
> or
> > >>>>> change log configs,  on a per operator basis (note this can be done
> > now
> > >>>> if
> > >>>>> you pass in a StateStoreSupplier, but this can be a bit
> cumbersome).
> > >>>>>
> > >>>>> So this is a bit of an open question. How can we change the DSL
> > >> overloads
> > >>>>> so that it flows, is simple to use and understand, and is easily
> > >> extended
> > >>>>> in the future?
> > >>>>>
> > >>>>> One option would be to use a fluent API approach for providing the
> > >>>> optional
> > >>>>> params, so something like this:
> > >>>>>
> > >>>>> groupedStream.count()
> > >>>>>    .withStoreName("name")
> > >>>>>    .withCachingEnabled(false)
> > >>>>>    .withLoggingEnabled(config)
> > >>>>>    .table()
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Another option would be to provide a Builder to the count method,
> so
> > it
> > >>>>> would look something like this:
> > >>>>> groupedStream.count(new
> > >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> > >>>>>
> > >>>>> Another option is to say: Hey we don't need this, what are you on
> > >> about!
> > >>>>>
> > >>>>> The above has focussed on state store related overloads, but the
> same
> > >>>> ideas
> > >>>>> could  be applied to joins etc, where we presently have many join
> > >> methods
> > >>>>> and many overloads.
> > >>>>>
> > >>>>> Anyway, i look forward to hearing your opinions.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Damian
> > >>>>>
> > >>>>
> > >>>
> > >>
> > >>
> > >
> >
> >
>
>
> --
> -- Guozhang
>
>

RE: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Kyle Winkelman <wi...@gmail.com>.
I like more of a builder pattern even though others have voiced against it. The reason I like it is because it makes it clear to the user that a call to KGroupedStream#count will return a KTable not some intermediate class that I need to undetstand. 
When trying to program in the fluent API that has been discussed most it feels difficult to know when you will actually get an object you can reuse. What if I make one KGroupedStream that I want to reuse, is it legal to reuse it or does this approach expect you to call grouped each time? This question doesn’t pop into my head at all in the builder pattern I assume I can reuse everything. 
Finally, I like .groupByKey and .groupBy(KeyValueMapper) not a big fan of the grouped.

Unfortunately, the below approach would require atleast 2 (probably 3) overloads (one for returning a KTable and one for returning a KTable with Windowed Key, probably would want to split windowed and sessionwindowed for ease of implementation) of each count, reduce, and aggregate.
Obviously not exhaustive but enough for you to get the picture. Count, Reduce, and Aggregate supply 3 static methods to initialize the builder:
// Count
KTable<String, Long> count = groupedStream.count(Count.count().withQueryableStoreName("my-store"));

// Windowed Count
KTable<Windowed<String>, Long> windowedCount = groupedStream.count(Count.windowed(TimeWindows.of(10L).until(10)).withQueryableStoreName("my-windowed-store"));

// Session Count
KTable<Windowed<String>, Long> sessionCount = groupedStream.count(Count.sessionWindowed(SessionWindows.with(10L)).withQueryableStoreName("my-session-windowed-store"));

// Reduce
Reducer<Long> reducer;
KTable<String, Long> reduce = groupedStream.reduce(reducer, Reduce.reduce().withQueryableStoreName("my-store"));

// Aggregate Windowed with Custom Store
Initializer<String> initializer;
Aggregator<String, Long, String> aggregator;
KTable<Windowed<String>, String> aggregate = groupedStream.aggregate(initializer, aggregator, Aggregate.windowed(TimeWindows.of(10L).until(10)).withStateStoreSupplier(stateStoreSupplier)));

// Cogroup SessionWindowed
KTable<String, String> cogrouped = groupedStream1.cogroup(aggregator1)
        .cogroup(groupedStream2, aggregator2)
        .aggregate(initializer, aggregator, Aggregate.sessionWindowed(SessionWindows.with(10L), sessionMerger).withQueryableStoreName("my-store"));



public class Count {

    public static class Windowed extends Count {
        private Windows windows;
    }
    public static class SessionWindowed extends Count {
        private SessionWindows sessionWindows;
    }

    public static Count count();
    public static Windowed windowed(Windows windows);
    public static SessionWindowed sessionWindowed(SessionWindows sessionWindows);

    // All withXXX(...) methods.
}

public class KGroupedStream {
    public KTable<K, Long> count(Count count);
    public KTable<Windowed<K>, Long> count(Count.Windowed count); 
    public KTable<Windowed<K>, Long> count(Count.SessionWindowed count);
…
}


Thanks,
Kyle

From: Guozhang Wang
Sent: Wednesday, June 28, 2017 7:45 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring

I played the current proposal a bit with https://github.com/dguy/kafka/
tree/dsl-experiment, and here are my observations:

1. Personally I prefer

    "stream.group(mapper) / stream.groupByKey()"

than

    "stream.group().withKeyMapper(mapper) / stream.group()"

Since 1) withKeyMapper is not enforced programmatically though it is not
"really" optional like others, 2) syntax-wise it reads more natural.

I think it is okay to add the APIs in (
https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java)
in KGroupedStream.


2. For the "withStateStoreSupplier" API, are the user supposed to pass in
the most-inner state store supplier (e.g. then one whose get() return
RocksDBStore), or it is supposed to return the most-outer supplier with
logging / metrics / etc? I think it would be more useful to only require
users pass in the inner state store supplier while specifying caching /
logging through other APIs.

In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we are
allowing users to call other APIs like "withQueryableName" multiple time,
but only call "withStateStoreSupplier" only once in the end. Why is that?


3. The current DSL seems to be only for aggregations, what about joins?


4. I think it is okay to keep the "withLogConfig": for the
StateStoreSupplier it will still be user code specifying the topology so I
do not see there is a big difference.


5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
windowed state store supplier to enforce typing?


Below are minor ones:

6. "withQueryableName": maybe better "withQueryableStateName"?

7. "withLogConfig": maybe better "withLoggingTopicConfig()"?



Guozhang



On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> I see your point about "when to add the processor to the topology". That
> is indeed an issue. Not sure it we could allow "updates" to the topology...
>
> I don't see any problem with having all the withXX() in KTable interface
> -- but this might be subjective.
>
>
> However, I don't understand your argument about putting aggregate()
> after the withXX() -- all the calls to withXX() set optional parameters
> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> be quite confusion for developers.
>
>
> -Matthias
>
> On 6/28/17 2:55 AM, Damian Guy wrote:
> >> I also think that mixing optional parameters with configs is a bad idea.
> >> Have not proposal for this atm but just wanted to mention it. Hope to
> >> find some time to come up with something.
> >>
> >>
> > Yes, i don't like the mix of config either. But the only real config here
> > is the logging config - which we don't really need as it can already be
> > done via a custom StateStoreSupplier.
> >
> >
> >> What I don't like in the current proposal is the
> >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> >> and .groupByKey() seems better. For clarity, we could rename to
> >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> >> some better names).
> >>
> >>
> > it could be groupByKey(), groupBy() or something different bt
> >
> >
> >
> >> The proposed pattern "chains" grouping and aggregation too close
> >> together. I would rather separate both more than less, ie, do into the
> >> opposite direction.
> >>
> >> I am also wondering, if we could so something more "fluent". The initial
> >> proposal was like:
> >>
> >>>> groupedStream.count()
> >>>>    .withStoreName("name")
> >>>>    .withCachingEnabled(false)
> >>>>    .withLoggingEnabled(config)
> >>>>    .table()
> >>
> >> The .table() statement in the end was kinda alien.
> >>
> >
> > I agree, but then all of the withXXX methods need to be on KTable which
> is
> > worse in my opinion. You also need something that is going to "build" the
> > internal processors and add them to the topology.
> >
> >
> >> The current proposal put the count() into the end -- ie, the optional
> >> parameter for count() have to specified on the .grouped() call -- this
> >> does not seems to be the best way either.
> >>
> >>
> > I actually prefer this method as you are building a grouped stream that
> you
> > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> etc
> > seems natural to me.
> >
> >
> >> I did not think this through in detail, but can't we just do the initial
> >> proposal with the .table() ?
> >>
> >> groupedStream.count().withStoreName("name").mapValues(...)
> >>
> >> Each .withXXX(...) return the current KTable and all the .withXXX() are
> >> just added to the KTable interface. Or do I miss anything why this wont'
> >> work or any obvious disadvantage?
> >>
> >>
> >>
> > See above.
> >
> >
> >>
> >> -Matthias
> >>
> >> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> >>> approach, but i think it is slightly nicer.
> >>> I agree with some of what Eno said about mixing configy stuff in the
> DSL,
> >>> but i think that enabling caching and enabling logging are things that
> >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> >>> though it is below) as this is actually config and we already have a
> way
> >> of
> >>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>> StateStoreSupplier for disabling caching etc, but as it stands that is
> a
> >>> bit of a tedious process for someone that just wants to use the default
> >>> storage engine, but not have caching enabled.
> >>>
> >>> There is also an orthogonal concern that Guozhang alluded to.... If you
> >>> want to plug in a custom storage engine and you want it to be logged
> etc,
> >>> you would currently need to implement that yourself. Ideally we can
> >> provide
> >>> a way where we will wrap the custom store with logging, metrics, etc. I
> >>> need to think about where this fits, it is probably more appropriate on
> >> the
> >>> Stores API.
> >>>
> >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>> // count with mapped key
> >>> final KTable<Long, Long> count = stream.grouped()
> >>>         .withKeyMapper(keyMapper)
> >>>         .withKeySerde(Serdes.Long())
> >>>         .withValueSerde(Serdes.String())
> >>>         .withQueryableName("my-store")
> >>>         .count();
> >>>
> >>> // windowed count
> >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .count();
> >>>
> >>> // windowed reduce
> >>> final Reducer<String> windowedReducer = null;
> >>> final KTable<Windowed<String>, String> windowedReduce =
> stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .reduce(windowedReducer);
> >>>
> >>> final Aggregator<String, String, Long> aggregator = null;
> >>> final Initializer<Long> init = null;
> >>>
> >>> // aggregate
> >>> final KTable<String, Long> aggregate = stream.grouped()
> >>>         .withQueryableName("my-aggregate-store")
> >>>         .aggregate(aggregator, init, Serdes.Long());
> >>>
> >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> stateStoreSupplier
> >> = null;
> >>>
> >>> // aggregate with custom store
> >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>         .aggregate(aggregator, init);
> >>>
> >>> // disable caching
> >>> stream.grouped()
> >>>         .withQueryableName("name")
> >>>         .withCachingEnabled(false)
> >>>         .count();
> >>>
> >>> // disable logging
> >>> stream.grouped()
> >>>         .withQueryableName("q")
> >>>         .withLoggingEnabled(false)
> >>>         .count();
> >>>
> >>> // override log config
> >>> final Reducer<String> reducer = null;
> >>> stream.grouped()
> >>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
> >>>         .reduce(reducer);
> >>>
> >>>
> >>> If anyone wants to play around with this you can find the code here:
> >>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>
> >>> Note: It won't actually work as most of the methods just return null.
> >>>
> >>> Thanks,
> >>> Damian
> >>>
> >>>
> >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> >>>
> >>>> Thanks Damian. I think both options have pros and cons. And both are
> >> better
> >>>> than overload abuse.
> >>>>
> >>>> The fluent API approach reads better, no mention of builder or build
> >>>> anywhere. The main downside is that the method signatures are a little
> >> less
> >>>> clear. By reading the method signature, one doesn't necessarily knows
> >> what
> >>>> it returns. Also, one needs to figure out the special method
> (`table()`
> >> in
> >>>> this case) that gives you what you actually care about (`KTable` in
> this
> >>>> case). Not major issues, but worth mentioning while doing the
> >> comparison.
> >>>>
> >>>> The builder approach avoids the issues mentioned above, but it doesn't
> >> read
> >>>> as well.
> >>>>
> >>>> Ismael
> >>>>
> >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> I'd like to get a discussion going around some of the API choices
> we've
> >>>>> made in the DLS. In particular those that relate to stateful
> operations
> >>>>> (though this could expand).
> >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> >> there
> >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> and
> >> i
> >>>>> feel it is only going to get worse as we add more optional params. In
> >>>>> particular we've had some requests to be able to turn caching off, or
> >>>>> change log configs,  on a per operator basis (note this can be done
> now
> >>>> if
> >>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>>>>
> >>>>> So this is a bit of an open question. How can we change the DSL
> >> overloads
> >>>>> so that it flows, is simple to use and understand, and is easily
> >> extended
> >>>>> in the future?
> >>>>>
> >>>>> One option would be to use a fluent API approach for providing the
> >>>> optional
> >>>>> params, so something like this:
> >>>>>
> >>>>> groupedStream.count()
> >>>>>    .withStoreName("name")
> >>>>>    .withCachingEnabled(false)
> >>>>>    .withLoggingEnabled(config)
> >>>>>    .table()
> >>>>>
> >>>>>
> >>>>>
> >>>>> Another option would be to provide a Builder to the count method, so
> it
> >>>>> would look something like this:
> >>>>> groupedStream.count(new
> >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>
> >>>>> Another option is to say: Hey we don't need this, what are you on
> >> about!
> >>>>>
> >>>>> The above has focussed on state store related overloads, but the same
> >>>> ideas
> >>>>> could  be applied to joins etc, where we presently have many join
> >> methods
> >>>>> and many overloads.
> >>>>>
> >>>>> Anyway, i look forward to hearing your opinions.
> >>>>>
> >>>>> Thanks,
> >>>>> Damian
> >>>>>
> >>>>
> >>>
> >>
> >>
> >
>
>


-- 
-- Guozhang


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Guozhang Wang <wa...@gmail.com>.
I played the current proposal a bit with https://github.com/dguy/kafka/
tree/dsl-experiment, and here are my observations:

1. Personally I prefer

    "stream.group(mapper) / stream.groupByKey()"

than

    "stream.group().withKeyMapper(mapper) / stream.group()"

Since 1) withKeyMapper is not enforced programmatically though it is not
"really" optional like others, 2) syntax-wise it reads more natural.

I think it is okay to add the APIs in (
https://github.com/dguy/kafka/blob/dsl-experiment/streams/src/main/java/org/apache/kafka/streams/kstream/GroupedStream.java)
in KGroupedStream.


2. For the "withStateStoreSupplier" API, are the user supposed to pass in
the most-inner state store supplier (e.g. then one whose get() return
RocksDBStore), or it is supposed to return the most-outer supplier with
logging / metrics / etc? I think it would be more useful to only require
users pass in the inner state store supplier while specifying caching /
logging through other APIs.

In addition, the "GroupedWithCustomStore" is a bit suspicious to me: we are
allowing users to call other APIs like "withQueryableName" multiple time,
but only call "withStateStoreSupplier" only once in the end. Why is that?


3. The current DSL seems to be only for aggregations, what about joins?


4. I think it is okay to keep the "withLogConfig": for the
StateStoreSupplier it will still be user code specifying the topology so I
do not see there is a big difference.


5. "WindowedGroupedStream" 's withStateStoreSupplier should take the
windowed state store supplier to enforce typing?


Below are minor ones:

6. "withQueryableName": maybe better "withQueryableStateName"?

7. "withLogConfig": maybe better "withLoggingTopicConfig()"?



Guozhang



On Wed, Jun 28, 2017 at 3:59 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> I see your point about "when to add the processor to the topology". That
> is indeed an issue. Not sure it we could allow "updates" to the topology...
>
> I don't see any problem with having all the withXX() in KTable interface
> -- but this might be subjective.
>
>
> However, I don't understand your argument about putting aggregate()
> after the withXX() -- all the calls to withXX() set optional parameters
> for aggregate() and not for groupBy() -- but a groupBy().withXX()
> indicates that the withXX() belongs to the groupBy(). IMHO, this might
> be quite confusion for developers.
>
>
> -Matthias
>
> On 6/28/17 2:55 AM, Damian Guy wrote:
> >> I also think that mixing optional parameters with configs is a bad idea.
> >> Have not proposal for this atm but just wanted to mention it. Hope to
> >> find some time to come up with something.
> >>
> >>
> > Yes, i don't like the mix of config either. But the only real config here
> > is the logging config - which we don't really need as it can already be
> > done via a custom StateStoreSupplier.
> >
> >
> >> What I don't like in the current proposal is the
> >> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> >> and .groupByKey() seems better. For clarity, we could rename to
> >> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> >> some better names).
> >>
> >>
> > it could be groupByKey(), groupBy() or something different bt
> >
> >
> >
> >> The proposed pattern "chains" grouping and aggregation too close
> >> together. I would rather separate both more than less, ie, do into the
> >> opposite direction.
> >>
> >> I am also wondering, if we could so something more "fluent". The initial
> >> proposal was like:
> >>
> >>>> groupedStream.count()
> >>>>    .withStoreName("name")
> >>>>    .withCachingEnabled(false)
> >>>>    .withLoggingEnabled(config)
> >>>>    .table()
> >>
> >> The .table() statement in the end was kinda alien.
> >>
> >
> > I agree, but then all of the withXXX methods need to be on KTable which
> is
> > worse in my opinion. You also need something that is going to "build" the
> > internal processors and add them to the topology.
> >
> >
> >> The current proposal put the count() into the end -- ie, the optional
> >> parameter for count() have to specified on the .grouped() call -- this
> >> does not seems to be the best way either.
> >>
> >>
> > I actually prefer this method as you are building a grouped stream that
> you
> > will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..)
> etc
> > seems natural to me.
> >
> >
> >> I did not think this through in detail, but can't we just do the initial
> >> proposal with the .table() ?
> >>
> >> groupedStream.count().withStoreName("name").mapValues(...)
> >>
> >> Each .withXXX(...) return the current KTable and all the .withXXX() are
> >> just added to the KTable interface. Or do I miss anything why this wont'
> >> work or any obvious disadvantage?
> >>
> >>
> >>
> > See above.
> >
> >
> >>
> >> -Matthias
> >>
> >> On 6/22/17 4:06 AM, Damian Guy wrote:
> >>> Thanks everyone. My latest attempt is below. It builds on the fluent
> >>> approach, but i think it is slightly nicer.
> >>> I agree with some of what Eno said about mixing configy stuff in the
> DSL,
> >>> but i think that enabling caching and enabling logging are things that
> >>> aren't actually config. I'd probably not add withLogConfig(...) (even
> >>> though it is below) as this is actually config and we already have a
> way
> >> of
> >>> doing that, via the StateStoreSupplier. Arguably we could use the
> >>> StateStoreSupplier for disabling caching etc, but as it stands that is
> a
> >>> bit of a tedious process for someone that just wants to use the default
> >>> storage engine, but not have caching enabled.
> >>>
> >>> There is also an orthogonal concern that Guozhang alluded to.... If you
> >>> want to plug in a custom storage engine and you want it to be logged
> etc,
> >>> you would currently need to implement that yourself. Ideally we can
> >> provide
> >>> a way where we will wrap the custom store with logging, metrics, etc. I
> >>> need to think about where this fits, it is probably more appropriate on
> >> the
> >>> Stores API.
> >>>
> >>> final KeyValueMapper<String, String, Long> keyMapper = null;
> >>> // count with mapped key
> >>> final KTable<Long, Long> count = stream.grouped()
> >>>         .withKeyMapper(keyMapper)
> >>>         .withKeySerde(Serdes.Long())
> >>>         .withValueSerde(Serdes.String())
> >>>         .withQueryableName("my-store")
> >>>         .count();
> >>>
> >>> // windowed count
> >>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .count();
> >>>
> >>> // windowed reduce
> >>> final Reducer<String> windowedReducer = null;
> >>> final KTable<Windowed<String>, String> windowedReduce =
> stream.grouped()
> >>>         .withQueryableName("my-window-store")
> >>>         .windowed(TimeWindows.of(10L).until(10))
> >>>         .reduce(windowedReducer);
> >>>
> >>> final Aggregator<String, String, Long> aggregator = null;
> >>> final Initializer<Long> init = null;
> >>>
> >>> // aggregate
> >>> final KTable<String, Long> aggregate = stream.grouped()
> >>>         .withQueryableName("my-aggregate-store")
> >>>         .aggregate(aggregator, init, Serdes.Long());
> >>>
> >>> final StateStoreSupplier<KeyValueStore<String, Long>>
> stateStoreSupplier
> >> = null;
> >>>
> >>> // aggregate with custom store
> >>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >>>         .withStateStoreSupplier(stateStoreSupplier)
> >>>         .aggregate(aggregator, init);
> >>>
> >>> // disable caching
> >>> stream.grouped()
> >>>         .withQueryableName("name")
> >>>         .withCachingEnabled(false)
> >>>         .count();
> >>>
> >>> // disable logging
> >>> stream.grouped()
> >>>         .withQueryableName("q")
> >>>         .withLoggingEnabled(false)
> >>>         .count();
> >>>
> >>> // override log config
> >>> final Reducer<String> reducer = null;
> >>> stream.grouped()
> >>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
> >>>         .reduce(reducer);
> >>>
> >>>
> >>> If anyone wants to play around with this you can find the code here:
> >>> https://github.com/dguy/kafka/tree/dsl-experiment
> >>>
> >>> Note: It won't actually work as most of the methods just return null.
> >>>
> >>> Thanks,
> >>> Damian
> >>>
> >>>
> >>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> >>>
> >>>> Thanks Damian. I think both options have pros and cons. And both are
> >> better
> >>>> than overload abuse.
> >>>>
> >>>> The fluent API approach reads better, no mention of builder or build
> >>>> anywhere. The main downside is that the method signatures are a little
> >> less
> >>>> clear. By reading the method signature, one doesn't necessarily knows
> >> what
> >>>> it returns. Also, one needs to figure out the special method
> (`table()`
> >> in
> >>>> this case) that gives you what you actually care about (`KTable` in
> this
> >>>> case). Not major issues, but worth mentioning while doing the
> >> comparison.
> >>>>
> >>>> The builder approach avoids the issues mentioned above, but it doesn't
> >> read
> >>>> as well.
> >>>>
> >>>> Ismael
> >>>>
> >>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> >> wrote:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> I'd like to get a discussion going around some of the API choices
> we've
> >>>>> made in the DLS. In particular those that relate to stateful
> operations
> >>>>> (though this could expand).
> >>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
> >> there
> >>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy
> and
> >> i
> >>>>> feel it is only going to get worse as we add more optional params. In
> >>>>> particular we've had some requests to be able to turn caching off, or
> >>>>> change log configs,  on a per operator basis (note this can be done
> now
> >>>> if
> >>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>>>>
> >>>>> So this is a bit of an open question. How can we change the DSL
> >> overloads
> >>>>> so that it flows, is simple to use and understand, and is easily
> >> extended
> >>>>> in the future?
> >>>>>
> >>>>> One option would be to use a fluent API approach for providing the
> >>>> optional
> >>>>> params, so something like this:
> >>>>>
> >>>>> groupedStream.count()
> >>>>>    .withStoreName("name")
> >>>>>    .withCachingEnabled(false)
> >>>>>    .withLoggingEnabled(config)
> >>>>>    .table()
> >>>>>
> >>>>>
> >>>>>
> >>>>> Another option would be to provide a Builder to the count method, so
> it
> >>>>> would look something like this:
> >>>>> groupedStream.count(new
> >>>>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>>>
> >>>>> Another option is to say: Hey we don't need this, what are you on
> >> about!
> >>>>>
> >>>>> The above has focussed on state store related overloads, but the same
> >>>> ideas
> >>>>> could  be applied to joins etc, where we presently have many join
> >> methods
> >>>>> and many overloads.
> >>>>>
> >>>>> Anyway, i look forward to hearing your opinions.
> >>>>>
> >>>>> Thanks,
> >>>>> Damian
> >>>>>
> >>>>
> >>>
> >>
> >>
> >
>
>


-- 
-- Guozhang

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
I see your point about "when to add the processor to the topology". That
is indeed an issue. Not sure it we could allow "updates" to the topology...

I don't see any problem with having all the withXX() in KTable interface
-- but this might be subjective.


However, I don't understand your argument about putting aggregate()
after the withXX() -- all the calls to withXX() set optional parameters
for aggregate() and not for groupBy() -- but a groupBy().withXX()
indicates that the withXX() belongs to the groupBy(). IMHO, this might
be quite confusion for developers.


-Matthias

On 6/28/17 2:55 AM, Damian Guy wrote:
>> I also think that mixing optional parameters with configs is a bad idea.
>> Have not proposal for this atm but just wanted to mention it. Hope to
>> find some time to come up with something.
>>
>>
> Yes, i don't like the mix of config either. But the only real config here
> is the logging config - which we don't really need as it can already be
> done via a custom StateStoreSupplier.
> 
> 
>> What I don't like in the current proposal is the
>> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
>> and .groupByKey() seems better. For clarity, we could rename to
>> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
>> some better names).
>>
>>
> it could be groupByKey(), groupBy() or something different bt
> 
> 
> 
>> The proposed pattern "chains" grouping and aggregation too close
>> together. I would rather separate both more than less, ie, do into the
>> opposite direction.
>>
>> I am also wondering, if we could so something more "fluent". The initial
>> proposal was like:
>>
>>>> groupedStream.count()
>>>>    .withStoreName("name")
>>>>    .withCachingEnabled(false)
>>>>    .withLoggingEnabled(config)
>>>>    .table()
>>
>> The .table() statement in the end was kinda alien.
>>
> 
> I agree, but then all of the withXXX methods need to be on KTable which is
> worse in my opinion. You also need something that is going to "build" the
> internal processors and add them to the topology.
> 
> 
>> The current proposal put the count() into the end -- ie, the optional
>> parameter for count() have to specified on the .grouped() call -- this
>> does not seems to be the best way either.
>>
>>
> I actually prefer this method as you are building a grouped stream that you
> will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..) etc
> seems natural to me.
> 
> 
>> I did not think this through in detail, but can't we just do the initial
>> proposal with the .table() ?
>>
>> groupedStream.count().withStoreName("name").mapValues(...)
>>
>> Each .withXXX(...) return the current KTable and all the .withXXX() are
>> just added to the KTable interface. Or do I miss anything why this wont'
>> work or any obvious disadvantage?
>>
>>
>>
> See above.
> 
> 
>>
>> -Matthias
>>
>> On 6/22/17 4:06 AM, Damian Guy wrote:
>>> Thanks everyone. My latest attempt is below. It builds on the fluent
>>> approach, but i think it is slightly nicer.
>>> I agree with some of what Eno said about mixing configy stuff in the DSL,
>>> but i think that enabling caching and enabling logging are things that
>>> aren't actually config. I'd probably not add withLogConfig(...) (even
>>> though it is below) as this is actually config and we already have a way
>> of
>>> doing that, via the StateStoreSupplier. Arguably we could use the
>>> StateStoreSupplier for disabling caching etc, but as it stands that is a
>>> bit of a tedious process for someone that just wants to use the default
>>> storage engine, but not have caching enabled.
>>>
>>> There is also an orthogonal concern that Guozhang alluded to.... If you
>>> want to plug in a custom storage engine and you want it to be logged etc,
>>> you would currently need to implement that yourself. Ideally we can
>> provide
>>> a way where we will wrap the custom store with logging, metrics, etc. I
>>> need to think about where this fits, it is probably more appropriate on
>> the
>>> Stores API.
>>>
>>> final KeyValueMapper<String, String, Long> keyMapper = null;
>>> // count with mapped key
>>> final KTable<Long, Long> count = stream.grouped()
>>>         .withKeyMapper(keyMapper)
>>>         .withKeySerde(Serdes.Long())
>>>         .withValueSerde(Serdes.String())
>>>         .withQueryableName("my-store")
>>>         .count();
>>>
>>> // windowed count
>>> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
>>>         .withQueryableName("my-window-store")
>>>         .windowed(TimeWindows.of(10L).until(10))
>>>         .count();
>>>
>>> // windowed reduce
>>> final Reducer<String> windowedReducer = null;
>>> final KTable<Windowed<String>, String> windowedReduce = stream.grouped()
>>>         .withQueryableName("my-window-store")
>>>         .windowed(TimeWindows.of(10L).until(10))
>>>         .reduce(windowedReducer);
>>>
>>> final Aggregator<String, String, Long> aggregator = null;
>>> final Initializer<Long> init = null;
>>>
>>> // aggregate
>>> final KTable<String, Long> aggregate = stream.grouped()
>>>         .withQueryableName("my-aggregate-store")
>>>         .aggregate(aggregator, init, Serdes.Long());
>>>
>>> final StateStoreSupplier<KeyValueStore<String, Long>> stateStoreSupplier
>> = null;
>>>
>>> // aggregate with custom store
>>> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>>>         .withStateStoreSupplier(stateStoreSupplier)
>>>         .aggregate(aggregator, init);
>>>
>>> // disable caching
>>> stream.grouped()
>>>         .withQueryableName("name")
>>>         .withCachingEnabled(false)
>>>         .count();
>>>
>>> // disable logging
>>> stream.grouped()
>>>         .withQueryableName("q")
>>>         .withLoggingEnabled(false)
>>>         .count();
>>>
>>> // override log config
>>> final Reducer<String> reducer = null;
>>> stream.grouped()
>>>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
>>>         .reduce(reducer);
>>>
>>>
>>> If anyone wants to play around with this you can find the code here:
>>> https://github.com/dguy/kafka/tree/dsl-experiment
>>>
>>> Note: It won't actually work as most of the methods just return null.
>>>
>>> Thanks,
>>> Damian
>>>
>>>
>>> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
>>>
>>>> Thanks Damian. I think both options have pros and cons. And both are
>> better
>>>> than overload abuse.
>>>>
>>>> The fluent API approach reads better, no mention of builder or build
>>>> anywhere. The main downside is that the method signatures are a little
>> less
>>>> clear. By reading the method signature, one doesn't necessarily knows
>> what
>>>> it returns. Also, one needs to figure out the special method (`table()`
>> in
>>>> this case) that gives you what you actually care about (`KTable` in this
>>>> case). Not major issues, but worth mentioning while doing the
>> comparison.
>>>>
>>>> The builder approach avoids the issues mentioned above, but it doesn't
>> read
>>>> as well.
>>>>
>>>> Ismael
>>>>
>>>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
>> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> I'd like to get a discussion going around some of the API choices we've
>>>>> made in the DLS. In particular those that relate to stateful operations
>>>>> (though this could expand).
>>>>> As it stands we lean heavily on overloaded methods in the API, i.e,
>> there
>>>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
>> i
>>>>> feel it is only going to get worse as we add more optional params. In
>>>>> particular we've had some requests to be able to turn caching off, or
>>>>> change log configs,  on a per operator basis (note this can be done now
>>>> if
>>>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>>>
>>>>> So this is a bit of an open question. How can we change the DSL
>> overloads
>>>>> so that it flows, is simple to use and understand, and is easily
>> extended
>>>>> in the future?
>>>>>
>>>>> One option would be to use a fluent API approach for providing the
>>>> optional
>>>>> params, so something like this:
>>>>>
>>>>> groupedStream.count()
>>>>>    .withStoreName("name")
>>>>>    .withCachingEnabled(false)
>>>>>    .withLoggingEnabled(config)
>>>>>    .table()
>>>>>
>>>>>
>>>>>
>>>>> Another option would be to provide a Builder to the count method, so it
>>>>> would look something like this:
>>>>> groupedStream.count(new
>>>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>>>
>>>>> Another option is to say: Hey we don't need this, what are you on
>> about!
>>>>>
>>>>> The above has focussed on state store related overloads, but the same
>>>> ideas
>>>>> could  be applied to joins etc, where we presently have many join
>> methods
>>>>> and many overloads.
>>>>>
>>>>> Anyway, i look forward to hearing your opinions.
>>>>>
>>>>> Thanks,
>>>>> Damian
>>>>>
>>>>
>>>
>>
>>
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
> I also think that mixing optional parameters with configs is a bad idea.
> Have not proposal for this atm but just wanted to mention it. Hope to
> find some time to come up with something.
>
>
Yes, i don't like the mix of config either. But the only real config here
is the logging config - which we don't really need as it can already be
done via a custom StateStoreSupplier.


> What I don't like in the current proposal is the
> .grouped().withKeyMapper() -- the current solution with .groupBy(...)
> and .groupByKey() seems better. For clarity, we could rename to
> .groupByNewKey(...) and .groupByCurrentKey() (even if we should find
> some better names).
>
>
it could be groupByKey(), groupBy() or something different bt



> The proposed pattern "chains" grouping and aggregation too close
> together. I would rather separate both more than less, ie, do into the
> opposite direction.
>
> I am also wondering, if we could so something more "fluent". The initial
> proposal was like:
>
> >> groupedStream.count()
> >>    .withStoreName("name")
> >>    .withCachingEnabled(false)
> >>    .withLoggingEnabled(config)
> >>    .table()
>
> The .table() statement in the end was kinda alien.
>

I agree, but then all of the withXXX methods need to be on KTable which is
worse in my opinion. You also need something that is going to "build" the
internal processors and add them to the topology.


> The current proposal put the count() into the end -- ie, the optional
> parameter for count() have to specified on the .grouped() call -- this
> does not seems to be the best way either.
>
>
I actually prefer this method as you are building a grouped stream that you
will aggregate. So table.grouped(...).withOptionalStuff().aggregate(..) etc
seems natural to me.


> I did not think this through in detail, but can't we just do the initial
> proposal with the .table() ?
>
> groupedStream.count().withStoreName("name").mapValues(...)
>
> Each .withXXX(...) return the current KTable and all the .withXXX() are
> just added to the KTable interface. Or do I miss anything why this wont'
> work or any obvious disadvantage?
>
>
>
See above.


>
> -Matthias
>
> On 6/22/17 4:06 AM, Damian Guy wrote:
> > Thanks everyone. My latest attempt is below. It builds on the fluent
> > approach, but i think it is slightly nicer.
> > I agree with some of what Eno said about mixing configy stuff in the DSL,
> > but i think that enabling caching and enabling logging are things that
> > aren't actually config. I'd probably not add withLogConfig(...) (even
> > though it is below) as this is actually config and we already have a way
> of
> > doing that, via the StateStoreSupplier. Arguably we could use the
> > StateStoreSupplier for disabling caching etc, but as it stands that is a
> > bit of a tedious process for someone that just wants to use the default
> > storage engine, but not have caching enabled.
> >
> > There is also an orthogonal concern that Guozhang alluded to.... If you
> > want to plug in a custom storage engine and you want it to be logged etc,
> > you would currently need to implement that yourself. Ideally we can
> provide
> > a way where we will wrap the custom store with logging, metrics, etc. I
> > need to think about where this fits, it is probably more appropriate on
> the
> > Stores API.
> >
> > final KeyValueMapper<String, String, Long> keyMapper = null;
> > // count with mapped key
> > final KTable<Long, Long> count = stream.grouped()
> >         .withKeyMapper(keyMapper)
> >         .withKeySerde(Serdes.Long())
> >         .withValueSerde(Serdes.String())
> >         .withQueryableName("my-store")
> >         .count();
> >
> > // windowed count
> > final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
> >         .withQueryableName("my-window-store")
> >         .windowed(TimeWindows.of(10L).until(10))
> >         .count();
> >
> > // windowed reduce
> > final Reducer<String> windowedReducer = null;
> > final KTable<Windowed<String>, String> windowedReduce = stream.grouped()
> >         .withQueryableName("my-window-store")
> >         .windowed(TimeWindows.of(10L).until(10))
> >         .reduce(windowedReducer);
> >
> > final Aggregator<String, String, Long> aggregator = null;
> > final Initializer<Long> init = null;
> >
> > // aggregate
> > final KTable<String, Long> aggregate = stream.grouped()
> >         .withQueryableName("my-aggregate-store")
> >         .aggregate(aggregator, init, Serdes.Long());
> >
> > final StateStoreSupplier<KeyValueStore<String, Long>> stateStoreSupplier
> = null;
> >
> > // aggregate with custom store
> > final KTable<String, Long> aggWithCustomStore = stream.grouped()
> >         .withStateStoreSupplier(stateStoreSupplier)
> >         .aggregate(aggregator, init);
> >
> > // disable caching
> > stream.grouped()
> >         .withQueryableName("name")
> >         .withCachingEnabled(false)
> >         .count();
> >
> > // disable logging
> > stream.grouped()
> >         .withQueryableName("q")
> >         .withLoggingEnabled(false)
> >         .count();
> >
> > // override log config
> > final Reducer<String> reducer = null;
> > stream.grouped()
> >         .withLogConfig(Collections.singletonMap("segment.size", "10"))
> >         .reduce(reducer);
> >
> >
> > If anyone wants to play around with this you can find the code here:
> > https://github.com/dguy/kafka/tree/dsl-experiment
> >
> > Note: It won't actually work as most of the methods just return null.
> >
> > Thanks,
> > Damian
> >
> >
> > On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> >
> >> Thanks Damian. I think both options have pros and cons. And both are
> better
> >> than overload abuse.
> >>
> >> The fluent API approach reads better, no mention of builder or build
> >> anywhere. The main downside is that the method signatures are a little
> less
> >> clear. By reading the method signature, one doesn't necessarily knows
> what
> >> it returns. Also, one needs to figure out the special method (`table()`
> in
> >> this case) that gives you what you actually care about (`KTable` in this
> >> case). Not major issues, but worth mentioning while doing the
> comparison.
> >>
> >> The builder approach avoids the issues mentioned above, but it doesn't
> read
> >> as well.
> >>
> >> Ismael
> >>
> >> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com>
> wrote:
> >>
> >>> Hi,
> >>>
> >>> I'd like to get a discussion going around some of the API choices we've
> >>> made in the DLS. In particular those that relate to stateful operations
> >>> (though this could expand).
> >>> As it stands we lean heavily on overloaded methods in the API, i.e,
> there
> >>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and
> i
> >>> feel it is only going to get worse as we add more optional params. In
> >>> particular we've had some requests to be able to turn caching off, or
> >>> change log configs,  on a per operator basis (note this can be done now
> >> if
> >>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >>>
> >>> So this is a bit of an open question. How can we change the DSL
> overloads
> >>> so that it flows, is simple to use and understand, and is easily
> extended
> >>> in the future?
> >>>
> >>> One option would be to use a fluent API approach for providing the
> >> optional
> >>> params, so something like this:
> >>>
> >>> groupedStream.count()
> >>>    .withStoreName("name")
> >>>    .withCachingEnabled(false)
> >>>    .withLoggingEnabled(config)
> >>>    .table()
> >>>
> >>>
> >>>
> >>> Another option would be to provide a Builder to the count method, so it
> >>> would look something like this:
> >>> groupedStream.count(new
> >>> CountBuilder("storeName").withCachingEnabled(false).build())
> >>>
> >>> Another option is to say: Hey we don't need this, what are you on
> about!
> >>>
> >>> The above has focussed on state store related overloads, but the same
> >> ideas
> >>> could  be applied to joins etc, where we presently have many join
> methods
> >>> and many overloads.
> >>>
> >>> Anyway, i look forward to hearing your opinions.
> >>>
> >>> Thanks,
> >>> Damian
> >>>
> >>
> >
>
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Damian,

thanks for starting this discussion.

I am not a fan of the builder pattern. It's too clumsy to use IMHO an
raised the entry level bar.

I also think that mixing optional parameters with configs is a bad idea.
Have not proposal for this atm but just wanted to mention it. Hope to
find some time to come up with something.

What I don't like in the current proposal is the
.grouped().withKeyMapper() -- the current solution with .groupBy(...)
and .groupByKey() seems better. For clarity, we could rename to
.groupByNewKey(...) and .groupByCurrentKey() (even if we should find
some better names).

The proposed pattern "chains" grouping and aggregation too close
together. I would rather separate both more than less, ie, do into the
opposite direction.

I am also wondering, if we could so something more "fluent". The initial
proposal was like:

>> groupedStream.count()
>>    .withStoreName("name")
>>    .withCachingEnabled(false)
>>    .withLoggingEnabled(config)
>>    .table()

The .table() statement in the end was kinda alien.

The current proposal put the count() into the end -- ie, the optional
parameter for count() have to specified on the .grouped() call -- this
does not seems to be the best way either.

I did not think this through in detail, but can't we just do the initial
proposal with the .table() ?

groupedStream.count().withStoreName("name").mapValues(...)

Each .withXXX(...) return the current KTable and all the .withXXX() are
just added to the KTable interface. Or do I miss anything why this wont'
work or any obvious disadvantage?



-Matthias

On 6/22/17 4:06 AM, Damian Guy wrote:
> Thanks everyone. My latest attempt is below. It builds on the fluent
> approach, but i think it is slightly nicer.
> I agree with some of what Eno said about mixing configy stuff in the DSL,
> but i think that enabling caching and enabling logging are things that
> aren't actually config. I'd probably not add withLogConfig(...) (even
> though it is below) as this is actually config and we already have a way of
> doing that, via the StateStoreSupplier. Arguably we could use the
> StateStoreSupplier for disabling caching etc, but as it stands that is a
> bit of a tedious process for someone that just wants to use the default
> storage engine, but not have caching enabled.
> 
> There is also an orthogonal concern that Guozhang alluded to.... If you
> want to plug in a custom storage engine and you want it to be logged etc,
> you would currently need to implement that yourself. Ideally we can provide
> a way where we will wrap the custom store with logging, metrics, etc. I
> need to think about where this fits, it is probably more appropriate on the
> Stores API.
> 
> final KeyValueMapper<String, String, Long> keyMapper = null;
> // count with mapped key
> final KTable<Long, Long> count = stream.grouped()
>         .withKeyMapper(keyMapper)
>         .withKeySerde(Serdes.Long())
>         .withValueSerde(Serdes.String())
>         .withQueryableName("my-store")
>         .count();
> 
> // windowed count
> final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
>         .withQueryableName("my-window-store")
>         .windowed(TimeWindows.of(10L).until(10))
>         .count();
> 
> // windowed reduce
> final Reducer<String> windowedReducer = null;
> final KTable<Windowed<String>, String> windowedReduce = stream.grouped()
>         .withQueryableName("my-window-store")
>         .windowed(TimeWindows.of(10L).until(10))
>         .reduce(windowedReducer);
> 
> final Aggregator<String, String, Long> aggregator = null;
> final Initializer<Long> init = null;
> 
> // aggregate
> final KTable<String, Long> aggregate = stream.grouped()
>         .withQueryableName("my-aggregate-store")
>         .aggregate(aggregator, init, Serdes.Long());
> 
> final StateStoreSupplier<KeyValueStore<String, Long>> stateStoreSupplier = null;
> 
> // aggregate with custom store
> final KTable<String, Long> aggWithCustomStore = stream.grouped()
>         .withStateStoreSupplier(stateStoreSupplier)
>         .aggregate(aggregator, init);
> 
> // disable caching
> stream.grouped()
>         .withQueryableName("name")
>         .withCachingEnabled(false)
>         .count();
> 
> // disable logging
> stream.grouped()
>         .withQueryableName("q")
>         .withLoggingEnabled(false)
>         .count();
> 
> // override log config
> final Reducer<String> reducer = null;
> stream.grouped()
>         .withLogConfig(Collections.singletonMap("segment.size", "10"))
>         .reduce(reducer);
> 
> 
> If anyone wants to play around with this you can find the code here:
> https://github.com/dguy/kafka/tree/dsl-experiment
> 
> Note: It won't actually work as most of the methods just return null.
> 
> Thanks,
> Damian
> 
> 
> On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:
> 
>> Thanks Damian. I think both options have pros and cons. And both are better
>> than overload abuse.
>>
>> The fluent API approach reads better, no mention of builder or build
>> anywhere. The main downside is that the method signatures are a little less
>> clear. By reading the method signature, one doesn't necessarily knows what
>> it returns. Also, one needs to figure out the special method (`table()` in
>> this case) that gives you what you actually care about (`KTable` in this
>> case). Not major issues, but worth mentioning while doing the comparison.
>>
>> The builder approach avoids the issues mentioned above, but it doesn't read
>> as well.
>>
>> Ismael
>>
>> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>>
>>> Hi,
>>>
>>> I'd like to get a discussion going around some of the API choices we've
>>> made in the DLS. In particular those that relate to stateful operations
>>> (though this could expand).
>>> As it stands we lean heavily on overloaded methods in the API, i.e, there
>>> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
>>> feel it is only going to get worse as we add more optional params. In
>>> particular we've had some requests to be able to turn caching off, or
>>> change log configs,  on a per operator basis (note this can be done now
>> if
>>> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>>>
>>> So this is a bit of an open question. How can we change the DSL overloads
>>> so that it flows, is simple to use and understand, and is easily extended
>>> in the future?
>>>
>>> One option would be to use a fluent API approach for providing the
>> optional
>>> params, so something like this:
>>>
>>> groupedStream.count()
>>>    .withStoreName("name")
>>>    .withCachingEnabled(false)
>>>    .withLoggingEnabled(config)
>>>    .table()
>>>
>>>
>>>
>>> Another option would be to provide a Builder to the count method, so it
>>> would look something like this:
>>> groupedStream.count(new
>>> CountBuilder("storeName").withCachingEnabled(false).build())
>>>
>>> Another option is to say: Hey we don't need this, what are you on about!
>>>
>>> The above has focussed on state store related overloads, but the same
>> ideas
>>> could  be applied to joins etc, where we presently have many join methods
>>> and many overloads.
>>>
>>> Anyway, i look forward to hearing your opinions.
>>>
>>> Thanks,
>>> Damian
>>>
>>
> 


Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Damian Guy <da...@gmail.com>.
Thanks everyone. My latest attempt is below. It builds on the fluent
approach, but i think it is slightly nicer.
I agree with some of what Eno said about mixing configy stuff in the DSL,
but i think that enabling caching and enabling logging are things that
aren't actually config. I'd probably not add withLogConfig(...) (even
though it is below) as this is actually config and we already have a way of
doing that, via the StateStoreSupplier. Arguably we could use the
StateStoreSupplier for disabling caching etc, but as it stands that is a
bit of a tedious process for someone that just wants to use the default
storage engine, but not have caching enabled.

There is also an orthogonal concern that Guozhang alluded to.... If you
want to plug in a custom storage engine and you want it to be logged etc,
you would currently need to implement that yourself. Ideally we can provide
a way where we will wrap the custom store with logging, metrics, etc. I
need to think about where this fits, it is probably more appropriate on the
Stores API.

final KeyValueMapper<String, String, Long> keyMapper = null;
// count with mapped key
final KTable<Long, Long> count = stream.grouped()
        .withKeyMapper(keyMapper)
        .withKeySerde(Serdes.Long())
        .withValueSerde(Serdes.String())
        .withQueryableName("my-store")
        .count();

// windowed count
final KTable<Windowed<String>, Long> windowedCount = stream.grouped()
        .withQueryableName("my-window-store")
        .windowed(TimeWindows.of(10L).until(10))
        .count();

// windowed reduce
final Reducer<String> windowedReducer = null;
final KTable<Windowed<String>, String> windowedReduce = stream.grouped()
        .withQueryableName("my-window-store")
        .windowed(TimeWindows.of(10L).until(10))
        .reduce(windowedReducer);

final Aggregator<String, String, Long> aggregator = null;
final Initializer<Long> init = null;

// aggregate
final KTable<String, Long> aggregate = stream.grouped()
        .withQueryableName("my-aggregate-store")
        .aggregate(aggregator, init, Serdes.Long());

final StateStoreSupplier<KeyValueStore<String, Long>> stateStoreSupplier = null;

// aggregate with custom store
final KTable<String, Long> aggWithCustomStore = stream.grouped()
        .withStateStoreSupplier(stateStoreSupplier)
        .aggregate(aggregator, init);

// disable caching
stream.grouped()
        .withQueryableName("name")
        .withCachingEnabled(false)
        .count();

// disable logging
stream.grouped()
        .withQueryableName("q")
        .withLoggingEnabled(false)
        .count();

// override log config
final Reducer<String> reducer = null;
stream.grouped()
        .withLogConfig(Collections.singletonMap("segment.size", "10"))
        .reduce(reducer);


If anyone wants to play around with this you can find the code here:
https://github.com/dguy/kafka/tree/dsl-experiment

Note: It won't actually work as most of the methods just return null.

Thanks,
Damian


On Thu, 22 Jun 2017 at 11:18 Ismael Juma <is...@juma.me.uk> wrote:

> Thanks Damian. I think both options have pros and cons. And both are better
> than overload abuse.
>
> The fluent API approach reads better, no mention of builder or build
> anywhere. The main downside is that the method signatures are a little less
> clear. By reading the method signature, one doesn't necessarily knows what
> it returns. Also, one needs to figure out the special method (`table()` in
> this case) that gives you what you actually care about (`KTable` in this
> case). Not major issues, but worth mentioning while doing the comparison.
>
> The builder approach avoids the issues mentioned above, but it doesn't read
> as well.
>
> Ismael
>
> On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com> wrote:
>
> > Hi,
> >
> > I'd like to get a discussion going around some of the API choices we've
> > made in the DLS. In particular those that relate to stateful operations
> > (though this could expand).
> > As it stands we lean heavily on overloaded methods in the API, i.e, there
> > are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> > feel it is only going to get worse as we add more optional params. In
> > particular we've had some requests to be able to turn caching off, or
> > change log configs,  on a per operator basis (note this can be done now
> if
> > you pass in a StateStoreSupplier, but this can be a bit cumbersome).
> >
> > So this is a bit of an open question. How can we change the DSL overloads
> > so that it flows, is simple to use and understand, and is easily extended
> > in the future?
> >
> > One option would be to use a fluent API approach for providing the
> optional
> > params, so something like this:
> >
> > groupedStream.count()
> >    .withStoreName("name")
> >    .withCachingEnabled(false)
> >    .withLoggingEnabled(config)
> >    .table()
> >
> >
> >
> > Another option would be to provide a Builder to the count method, so it
> > would look something like this:
> > groupedStream.count(new
> > CountBuilder("storeName").withCachingEnabled(false).build())
> >
> > Another option is to say: Hey we don't need this, what are you on about!
> >
> > The above has focussed on state store related overloads, but the same
> ideas
> > could  be applied to joins etc, where we presently have many join methods
> > and many overloads.
> >
> > Anyway, i look forward to hearing your opinions.
> >
> > Thanks,
> > Damian
> >
>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

Posted by Ismael Juma <is...@juma.me.uk>.
Thanks Damian. I think both options have pros and cons. And both are better
than overload abuse.

The fluent API approach reads better, no mention of builder or build
anywhere. The main downside is that the method signatures are a little less
clear. By reading the method signature, one doesn't necessarily knows what
it returns. Also, one needs to figure out the special method (`table()` in
this case) that gives you what you actually care about (`KTable` in this
case). Not major issues, but worth mentioning while doing the comparison.

The builder approach avoids the issues mentioned above, but it doesn't read
as well.

Ismael

On Wed, Jun 21, 2017 at 3:37 PM, Damian Guy <da...@gmail.com> wrote:

> Hi,
>
> I'd like to get a discussion going around some of the API choices we've
> made in the DLS. In particular those that relate to stateful operations
> (though this could expand).
> As it stands we lean heavily on overloaded methods in the API, i.e, there
> are 9 overloads for KGroupedStream.count(..)! It is becoming noisy and i
> feel it is only going to get worse as we add more optional params. In
> particular we've had some requests to be able to turn caching off, or
> change log configs,  on a per operator basis (note this can be done now if
> you pass in a StateStoreSupplier, but this can be a bit cumbersome).
>
> So this is a bit of an open question. How can we change the DSL overloads
> so that it flows, is simple to use and understand, and is easily extended
> in the future?
>
> One option would be to use a fluent API approach for providing the optional
> params, so something like this:
>
> groupedStream.count()
>    .withStoreName("name")
>    .withCachingEnabled(false)
>    .withLoggingEnabled(config)
>    .table()
>
>
>
> Another option would be to provide a Builder to the count method, so it
> would look something like this:
> groupedStream.count(new
> CountBuilder("storeName").withCachingEnabled(false).build())
>
> Another option is to say: Hey we don't need this, what are you on about!
>
> The above has focussed on state store related overloads, but the same ideas
> could  be applied to joins etc, where we presently have many join methods
> and many overloads.
>
> Anyway, i look forward to hearing your opinions.
>
> Thanks,
> Damian
>