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....@gmail.com
<mailto:damian....@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.filip...@trivago.com
<mailto:jan.filip...@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....@gmail.com
<mailto:damian....@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
>
>