Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-11 Thread Jan Filipiak
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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-08 Thread Guozhang Wang
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 inter

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-04 Thread Jan Filipiak
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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-02 Thread Guozhang Wang
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 unne

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-02 Thread Jan Filipiak
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 overloa

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-02 Thread Bill Bejeck
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 s

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-02 Thread Damian Guy
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 wrote: > Hi all, > > after some further discussions, the best thing to show my Idea of

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-08-01 Thread Jan Filipiak
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. I

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-24 Thread Jan Filipiak
“my-store” already exists. Yes i agree and i think using builders is my preferred pattern. Cheers, Damian

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-19 Thread Guozhang Wang
winkelman.k...@gmail.com >> > <mailto:winkelman.k...@gmail.com>> >> >> > >> > wrote: >> > >> > Hi Damian, >> &

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-19 Thread Damian Guy
effectively a no-op until a count, > > reduce, or aggregate. The way I > > > > see > > > > it > > > >

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-18 Thread Clemens Valiente
re but instead truly remains optional. Cheers Clemens From: Jan Filipiak 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 ea

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-18 Thread Jan Filipiak
rred pattern. Cheers, Damian Thanks, Kyle From: Damian Guy Sent: Thursday, June

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-12 Thread Guozhang Wang
gt;>> 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 >>>>>>>>>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-09 Thread Jan Filipiak
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 count = kGroupedStream.count(Count.count().withQueryableStoreName("my-store")); The kGroupedStream is reusable and if th

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-09 Thread Matthias J. Sax
>>>>>>>>> >>>>>>>>> >>>>>>>> Sure, it is currently called withLogConfig() as that is the only >>>>>>>> thing >>>>>>> that >>>>>>>> is really config. &g

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-08 Thread Jan Filipiak
ed sessionWindowed(SessionWindows sessionWindows); // All withXXX(...) methods. } public class KGroupedStream { public KTable count(Count count); public KTable, Long> count(Count.Windowed count); public KTable, Long> count(Count.SessionWindowed count); … } Thanks, Kyle Fr

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-07 Thread Matthias J. Sax
s >>>>>>> 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 >>>>

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-07 Thread Jan Filipiak
ount = 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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-07 Thread Guozhang Wang
> > Sure, a generic name might be ok. >> > > >> > > >> > > >> > > >> > >> -Matthias >> > >> >> > >> >> > >> >> > >> On 6/29/17 7:37 AM, Damian Guy wrote: >&g

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-07 Thread Guozhang Wang
gt; > 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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-04 Thread Damian Guy
ontinue 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 ma

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-07-04 Thread Jan Filipiak
re” 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: d...@kafka.apache.org Subject: Re: [DISCUSS] Streams DSL/StateStore Refactoring Hi Kyle, Thanks for your input. Really

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-30 Thread Damian Guy
eated > >> 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). > >> > > >

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-29 Thread Damian Guy
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/KSt

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-24 Thread Jan Filipiak
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 jo

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Jan Filipiak
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)

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-22 Thread Eno Thereska
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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Guozhang Wang
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: --

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
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 wrote: > > +1 for the approach outlined above by Eno. > > On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy wrote: > >> Thanks Eno. >> >> Yes i agree. We cou

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Bill Bejeck
+1 for the approach outlined above by Eno. On Wed, Jun 21, 2017 at 11:28 AM, Damian Guy 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

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Damian Guy
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 Ther

Re: [DISCUSS] Streams DSL/StateStore Refactoring

2017-06-21 Thread Eno Thereska
(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 yo