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/01 09:03:00 UTC

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Hi Kyle,

Thanks for the update. I think just one initializer makes sense as it
should only be called once per key and generally it is just going to create
a new instance of whatever the Aggregate class is.

Cheers,
Damian

On Wed, 31 May 2017 at 20:09 Kyle Winkelman <wi...@gmail.com>
wrote:

> Hello all,
>
> I have spent some more time on this and the best alternative I have come up
> with is:
> KGroupedStream has a single cogroup call that takes an initializer and an
> aggregator.
> CogroupedKStream has a cogroup call that takes additional groupedStream
> aggregator pairs.
> CogroupedKStream has multiple aggregate methods that create the different
> stores.
>
> I plan on updating the kip but I want people's input on if we should have
> the initializer be passed in once at the beginning or if we should instead
> have the initializer be required for each call to one of the aggregate
> calls. The first makes more sense to me but doesnt allow the user to
> specify different initializers for different tables.
>
> Thanks,
> Kyle
>
> On May 24, 2017 7:46 PM, "Kyle Winkelman" <wi...@gmail.com>
> wrote:
>
> > Yea I really like that idea I'll see what I can do to update the kip and
> > my pr when I have some time. I'm not sure how well creating the
> > kstreamaggregates will go though because at that point I will have thrown
> > away the type of the values. It will be type safe I just may need to do a
> > little forcing.
> >
> > Thanks,
> > Kyle
> >
> > On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> >
> >> Kyle,
> >>
> >> Thanks for the explanations, my previous read on the wiki examples was
> >> wrong.
> >>
> >> So I guess my motivation should be "reduced" to: can we move the window
> >> specs param from "KGroupedStream#cogroup(..)" to
> >> "CogroupedKStream#aggregate(..)", and my motivations are:
> >>
> >> 1. minor: we can reduce the #.generics in CogroupedKStream from 3 to 2.
> >> 2. major: this is for extensibility of the APIs, and since we are
> removing
> >> the "Evolving" annotations on Streams it may be harder to change it
> again
> >> in the future. The extended use cases are that people wanted to have
> >> windowed running aggregates on different granularities, e.g. "give me
> the
> >> counts per-minute, per-hour, per-day and per-week", and today in DSL we
> >> need to specify that case in multiple aggregate operators, which gets a
> >> state store / changelog, etc. And it is possible to optimize it as well
> to
> >> a single state store. Its implementation would be tricky as you need to
> >> contain different lengthed windows within your window store but just
> from
> >> the public API point of view, it could be specified as:
> >>
> >> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >> "state-store-name");
> >>
> >> table1 = stream.aggregate(/*per-minute window*/)
> >> table2 = stream.aggregate(/*per-hour window*/)
> >> table3 = stream.aggregate(/*per-day window*/)
> >>
> >> while underlying we are only using a single store "state-store-name" for
> >> it.
> >>
> >>
> >> Although this feature is out of the scope of this KIP, I'd like to
> discuss
> >> if we can "leave the door open" to make such changes without modifying
> the
> >> public APIs .
> >>
> >> Guozhang
> >>
> >>
> >> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> winkelman.kyle@gmail.com
> >> >
> >> wrote:
> >>
> >> > I allow defining a single window/sessionwindow one time when you make
> >> the
> >> > cogroup call from a KGroupedStream. From then on you are using the
> >> cogroup
> >> > call from with in CogroupedKStream which doesnt accept any additional
> >> > windows/sessionwindows.
> >> >
> >> > Is this what you meant by your question or did I misunderstand?
> >> >
> >> > On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> >> >
> >> > Another question that came to me is on "window alignment": from the
> KIP
> >> it
> >> > seems you are allowing users to specify a (potentially different)
> window
> >> > spec in each co-grouped input stream. So if these window specs are
> >> > different how should we "align" them with different input streams? I
> >> think
> >> > it is more natural to only specify on window spec in the
> >> >
> >> > KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >> >
> >> >
> >> > And remove it from the cogroup() functions. WDYT?
> >> >
> >> >
> >> > Guozhang
> >> >
> >> > On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >> >
> >> > > Thanks for the proposal Kyle, this is a quite common use case to
> >> support
> >> > > such multi-way table join (i.e. N source tables with N aggregate
> func)
> >> > with
> >> > > a single store and N+1 serdes, I have seen lots of people using the
> >> > > low-level PAPI to achieve this goal.
> >> > >
> >> > >
> >> > > On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >> > winkelman.kyle@gmail.com
> >> > > > wrote:
> >> > >
> >> > >> I like your point about not handling other cases such as count and
> >> > reduce.
> >> > >>
> >> > >> I think that reduce may not make sense because reduce assumes that
> >> the
> >> > >> input values are the same as the output values. With cogroup there
> >> may
> >> > be
> >> > >> multiple different input types and then your output type cant be
> >> > multiple
> >> > >> different things. In the case where you have all matching value
> types
> >> > you
> >> > >> can do KStreamBuilder#merge followed by the reduce.
> >> > >>
> >> > >> As for count I think it is possible to call count on all the
> >> individual
> >> > >> grouped streams and then do joins. Otherwise we could maybe make a
> >> > special
> >> > >> call in groupedstream for this case. Because in this case we dont
> >> need
> >> > to
> >> > >> do type checking on the values. It could be similar to the current
> >> count
> >> > >> methods but accept a var args of additonal grouped streams as well
> >> and
> >> > >> make
> >> > >> sure they have a key type of K.
> >> > >>
> >> > >> The way I have put the kip together is to ensure that we do type
> >> > checking.
> >> > >> I don't see a way we could group them all first and then make a
> call
> >> to
> >> > >> count, reduce, or aggregate because with aggregate they would need
> to
> >> > pass
> >> > >> a list of aggregators and we would have no way of type checking
> that
> >> > they
> >> > >> match the grouped streams.
> >> > >>
> >> > >> Thanks,
> >> > >> Kyle
> >> > >>
> >> > >> On May 19, 2017 11:42 AM, "Xavier Léauté" <xa...@confluent.io>
> >> wrote:
> >> > >>
> >> > >> > Sorry to jump on this thread so late. I agree this is a very
> useful
> >> > >> > addition and wanted to provide an additional use-case and some
> more
> >> > >> > comments.
> >> > >> >
> >> > >> > This is actually a very common analytics use-case in the ad-tech
> >> > >> industry.
> >> > >> > The typical setup will have an auction stream, an impression
> >> stream,
> >> > >> and a
> >> > >> > click stream. Those three streams need to be combined to compute
> >> > >> aggregate
> >> > >> > statistics (e.g. impression statistics, and click-through rates),
> >> > since
> >> > >> > most of the attributes of interest are only present the auction
> >> > stream.
> >> > >> >
> >> > >> > A simple way to do this is to co-group all the streams by the
> >> auction
> >> > >> key,
> >> > >> > and process updates to the co-group as events for each stream
> come
> >> in,
> >> > >> > keeping only one value from each stream before sending downstream
> >> for
> >> > >> > further processing / aggregation.
> >> > >> >
> >> > >> > One could view the result of that co-group operation as a
> "KTable"
> >> > with
> >> > >> > multiple values per key. The key being the grouping key, and the
> >> > values
> >> > >> > consisting of one value per stream.
> >> > >> >
> >> > >> > What I like about Kyle's approach is that allows elegant
> >> co-grouping
> >> > of
> >> > >> > multiple streams without having to worry about the number of
> >> streams,
> >> > >> and
> >> > >> > avoids dealing with Tuple types or other generic interfaces that
> >> could
> >> > >> get
> >> > >> > messy if we wanted to preserve all the value types in the
> resulting
> >> > >> > co-grouped stream.
> >> > >> >
> >> > >> > My only concern is that we only allow the cogroup + aggregate
> >> combined
> >> > >> > operation. This forces the user to build their own tuple
> >> serialization
> >> > >> > format if they want to preserve the individual input stream
> values
> >> as
> >> > a
> >> > >> > group. It also deviates quite a bit from our approach in
> >> > KGroupedStream
> >> > >> > which offers other operations, such as count and reduce, which
> >> should
> >> > >> also
> >> > >> > be applicable to a co-grouped stream.
> >> > >> >
> >> > >> > Overall I still think this is a really useful addition, but I
> feel
> >> we
> >> > >> > haven't spend much time trying to explore alternative DSLs that
> >> could
> >> > >> maybe
> >> > >> > generalize better or match our existing syntax more closely.
> >> > >> >
> >> > >> > On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> >> > winkelman.kyle@gmail.com
> >> > >> >
> >> > >> > wrote:
> >> > >> >
> >> > >> > > Eno, is there anyone else that is an expert in the kafka
> streams
> >> > realm
> >> > >> > that
> >> > >> > > I should reach out to for input?
> >> > >> > >
> >> > >> > > I believe Damian Guy is still planning on reviewing this more
> in
> >> > depth
> >> > >> > so I
> >> > >> > > will wait for his inputs before continuing.
> >> > >> > >
> >> > >> > > On May 9, 2017 7:30 AM, "Eno Thereska" <eno.thereska@gmail.com
> >
> >> > >> wrote:
> >> > >> > >
> >> > >> > > > Thanks Kyle, good arguments.
> >> > >> > > >
> >> > >> > > > Eno
> >> > >> > > >
> >> > >> > > > > On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> >> > >> winkelman.kyle@gmail.com
> >> > >> > >
> >> > >> > > > wrote:
> >> > >> > > > >
> >> > >> > > > > *- minor: could you add an exact example (similar to what
> >> Jay’s
> >> > >> > example
> >> > >> > > > is,
> >> > >> > > > > or like your Spark/Pig pointers had) to make this super
> >> > concrete?*
> >> > >> > > > > I have added a more concrete example to the KIP.
> >> > >> > > > >
> >> > >> > > > > *- my main concern is that we’re exposing this optimization
> >> to
> >> > the
> >> > >> > DSL.
> >> > >> > > > In
> >> > >> > > > > an ideal world, an optimizer would take the existing DSL
> and
> >> do
> >> > >> the
> >> > >> > > right
> >> > >> > > > > thing under the covers (create just one state store,
> arrange
> >> the
> >> > >> > nodes
> >> > >> > > > > etc). The original DSL had a bunch of small, composable
> >> pieces
> >> > >> > (group,
> >> > >> > > > > aggregate, join) that this proposal groups together. I’d
> >> like to
> >> > >> hear
> >> > >> > > > your
> >> > >> > > > > thoughts on whether it’s possible to do this optimization
> >> with
> >> > the
> >> > >> > > > current
> >> > >> > > > > DSL, at the topology builder level.*
> >> > >> > > > > You would have to make a lot of checks to understand if it
> is
> >> > even
> >> > >> > > > possible
> >> > >> > > > > to make this optimization:
> >> > >> > > > > 1. Make sure they are all KTableKTableOuterJoins
> >> > >> > > > > 2. None of the intermediate KTables are used for anything
> >> else.
> >> > >> > > > > 3. None of the intermediate stores are used. (This may be
> >> > >> impossible
> >> > >> > > > > especially if they use KafkaStreams#store after the
> topology
> >> has
> >> > >> > > already
> >> > >> > > > > been built.)
> >> > >> > > > > You would then need to make decisions during the
> >> optimization:
> >> > >> > > > > 1. Your new initializer would the composite of all the
> >> > individual
> >> > >> > > > > initializers and the valueJoiners.
> >> > >> > > > > 2. I am having a hard time thinking about how you would
> turn
> >> the
> >> > >> > > > > aggregators and valueJoiners into an aggregator that would
> >> work
> >> > on
> >> > >> > the
> >> > >> > > > > final object, but this may be possible.
> >> > >> > > > > 3. Which state store would you use? The ones declared would
> >> be
> >> > for
> >> > >> > the
> >> > >> > > > > aggregate values. None of the declared ones would be
> >> guaranteed
> >> > to
> >> > >> > hold
> >> > >> > > > the
> >> > >> > > > > final object. This would mean you must created a new state
> >> store
> >> > >> and
> >> > >> > > not
> >> > >> > > > > created any of the declared ones.
> >> > >> > > > >
> >> > >> > > > > The main argument I have against it is even if it could be
> >> done
> >> > I
> >> > >> > don't
> >> > >> > > > > know that we would want to have this be an optimization in
> >> the
> >> > >> > > background
> >> > >> > > > > because the user would still be required to think about all
> >> of
> >> > the
> >> > >> > > > > intermediate values that they shouldn't need to worry about
> >> if
> >> > >> they
> >> > >> > > only
> >> > >> > > > > care about the final object.
> >> > >> > > > >
> >> > >> > > > > In my opinion cogroup is a common enough case that it
> should
> >> be
> >> > >> part
> >> > >> > of
> >> > >> > > > the
> >> > >> > > > > composable pieces (group, aggregate, join) because we want
> to
> >> > >> allow
> >> > >> > > > people
> >> > >> > > > > to join more than 2 or more streams in an easy way. Right
> >> now I
> >> > >> don't
> >> > >> > > > think
> >> > >> > > > > we give them ways of handling this use case easily.
> >> > >> > > > >
> >> > >> > > > > *-I think there will be scope for several such
> optimizations
> >> in
> >> > >> the
> >> > >> > > > future
> >> > >> > > > > and perhaps at some point we need to think about decoupling
> >> the
> >> > >> 1:1
> >> > >> > > > mapping
> >> > >> > > > > from the DSL into the physical topology.*
> >> > >> > > > > I would argue that cogroup is not just an optimization it
> is
> >> a
> >> > new
> >> > >> > way
> >> > >> > > > for
> >> > >> > > > > the users to look at accomplishing a problem that requires
> >> > >> multiple
> >> > >> > > > > streams. I may sound like a broken record but I don't think
> >> > users
> >> > >> > > should
> >> > >> > > > > have to build the N-1 intermediate tables and deal with
> their
> >> > >> > > > initializers,
> >> > >> > > > > serdes and stores if all they care about is the final
> object.
> >> > >> > > > > Now if for example someone uses cogroup but doesn't supply
> >> > >> additional
> >> > >> > > > > streams and aggregators this case is equivalent to a single
> >> > >> grouped
> >> > >> > > > stream
> >> > >> > > > > making an aggregate call. This case is what I view an
> >> > optimization
> >> > >> > as,
> >> > >> > > we
> >> > >> > > > > could remove the KStreamCogroup and act as if there was
> just
> >> a
> >> > >> call
> >> > >> > to
> >> > >> > > > > KGroupedStream#aggregate instead of calling
> >> > >> KGroupedStream#cogroup.
> >> > >> > (I
> >> > >> > > > > would prefer to just write a warning saying that this is
> not
> >> how
> >> > >> > > cogroup
> >> > >> > > > is
> >> > >> > > > > to be used.)
> >> > >> > > > >
> >> > >> > > > > Thanks,
> >> > >> > > > > Kyle
> >> > >> > > > >
> >> > >> > > > > On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> >> > >> eno.thereska@gmail.com
> >> > >> > >
> >> > >> > > > wrote:
> >> > >> > > > >
> >> > >> > > > >> Hi Kyle,
> >> > >> > > > >>
> >> > >> > > > >> Thanks for the KIP again. A couple of comments:
> >> > >> > > > >>
> >> > >> > > > >> - minor: could you add an exact example (similar to what
> >> Jay’s
> >> > >> > example
> >> > >> > > > is,
> >> > >> > > > >> or like your Spark/Pig pointers had) to make this super
> >> > concrete?
> >> > >> > > > >>
> >> > >> > > > >> - my main concern is that we’re exposing this optimization
> >> to
> >> > the
> >> > >> > DSL.
> >> > >> > > > In
> >> > >> > > > >> an ideal world, an optimizer would take the existing DSL
> >> and do
> >> > >> the
> >> > >> > > > right
> >> > >> > > > >> thing under the covers (create just one state store,
> arrange
> >> > the
> >> > >> > nodes
> >> > >> > > > >> etc). The original DSL had a bunch of small, composable
> >> pieces
> >> > >> > (group,
> >> > >> > > > >> aggregate, join) that this proposal groups together. I’d
> >> like
> >> > to
> >> > >> > hear
> >> > >> > > > your
> >> > >> > > > >> thoughts on whether it’s possible to do this optimization
> >> with
> >> > >> the
> >> > >> > > > current
> >> > >> > > > >> DSL, at the topology builder level.
> >> > >> > > > >>
> >> > >> > > > >> I think there will be scope for several such optimizations
> >> in
> >> > the
> >> > >> > > future
> >> > >> > > > >> and perhaps at some point we need to think about
> decoupling
> >> the
> >> > >> 1:1
> >> > >> > > > mapping
> >> > >> > > > >> from the DSL into the physical topology.
> >> > >> > > > >>
> >> > >> > > > >> Thanks
> >> > >> > > > >> Eno
> >> > >> > > > >>
> >> > >> > > > >>> On May 5, 2017, at 4:39 PM, Jay Kreps <ja...@confluent.io>
> >> > wrote:
> >> > >> > > > >>>
> >> > >> > > > >>> I haven't digested the proposal but the use case is
> pretty
> >> > >> common.
> >> > >> > An
> >> > >> > > > >>> example would be the "customer 360" or "unified customer
> >> > >> profile"
> >> > >> > use
> >> > >> > > > >> case
> >> > >> > > > >>> we often use. In that use case you have a dozen systems
> >> each
> >> > of
> >> > >> > which
> >> > >> > > > has
> >> > >> > > > >>> some information about your customer (account details,
> >> > settings,
> >> > >> > > > billing
> >> > >> > > > >>> info, customer service contacts, purchase history, etc).
> >> Your
> >> > >> goal
> >> > >> > is
> >> > >> > > > to
> >> > >> > > > >>> join/munge these into a single profile record for each
> >> > customer
> >> > >> > that
> >> > >> > > > has
> >> > >> > > > >>> all the relevant info in a usable form and is up-to-date
> >> with
> >> > >> all
> >> > >> > the
> >> > >> > > > >>> source systems. If you implement that with kstreams as a
> >> > >> sequence
> >> > >> > of
> >> > >> > > > >> joins
> >> > >> > > > >>> i think today we'd fully materialize N-1 intermediate
> >> tables.
> >> > >> But
> >> > >> > > > clearly
> >> > >> > > > >>> you only need a single stage to group all these things
> that
> >> > are
> >> > >> > > already
> >> > >> > > > >>> co-partitioned. A distributed database would do this
> under
> >> the
> >> > >> > covers
> >> > >> > > > >> which
> >> > >> > > > >>> is arguably better (at least when it does the right
> thing)
> >> and
> >> > >> > > perhaps
> >> > >> > > > we
> >> > >> > > > >>> could do the same thing but I'm not sure we know the
> >> > >> partitioning
> >> > >> > so
> >> > >> > > we
> >> > >> > > > >> may
> >> > >> > > > >>> need an explicit cogroup command that impllies they are
> >> > already
> >> > >> > > > >>> co-partitioned.
> >> > >> > > > >>>
> >> > >> > > > >>> -Jay
> >> > >> > > > >>>
> >> > >> > > > >>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> >> > >> > > > winkelman.kyle@gmail.com
> >> > >> > > > >>>
> >> > >> > > > >>> wrote:
> >> > >> > > > >>>
> >> > >> > > > >>>> Yea thats a good way to look at it.
> >> > >> > > > >>>> I have seen this type of functionality in a couple other
> >> > >> platforms
> >> > >> > > > like
> >> > >> > > > >>>> spark and pig.
> >> > >> > > > >>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> >> > >> > > > >> PairRDDFunctions.html
> >> > >> > > > >>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
> >> > >> > > > >> cogroup_operator.htm
> >> > >> > > > >>>>
> >> > >> > > > >>>>
> >> > >> > > > >>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> >> damian.guy@gmail.com>
> >> > >> > wrote:
> >> > >> > > > >>>>
> >> > >> > > > >>>>> Hi Kyle,
> >> > >> > > > >>>>>
> >> > >> > > > >>>>> If i'm reading this correctly it is like an N way outer
> >> > join?
> >> > >> So
> >> > >> > an
> >> > >> > > > >> input
> >> > >> > > > >>>>> on any stream will always produce a new aggregated
> value
> >> -
> >> > is
> >> > >> > that
> >> > >> > > > >>>> correct?
> >> > >> > > > >>>>> Effectively, each Aggregator just looks up the current
> >> > value,
> >> > >> > > > >> aggregates
> >> > >> > > > >>>>> and forwards the result.
> >> > >> > > > >>>>> I need to look into it and think about it a bit more,
> >> but it
> >> > >> > seems
> >> > >> > > > like
> >> > >> > > > >>>> it
> >> > >> > > > >>>>> could be a useful optimization.
> >> > >> > > > >>>>>
> >> > >> > > > >>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> >> > >> > > winkelman.kyle@gmail.com
> >> > >> > > > >
> >> > >> > > > >>>>> wrote:
> >> > >> > > > >>>>>
> >> > >> > > > >>>>>> I sure can. I have added the following description to
> my
> >> > >> KIP. If
> >> > >> > > > this
> >> > >> > > > >>>>>> doesn't help let me know and I will take some more
> time
> >> to
> >> > >> > build a
> >> > >> > > > >>>>> diagram
> >> > >> > > > >>>>>> and make more of a step by step description:
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> Example with Current API:
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> KTable<K, V1> table1 =
> >> > >> > > > >>>>>> builder.stream("topic1").groupByKey().aggregate(
> >> > initializer1
> >> > >> ,
> >> > >> > > > >>>>> aggregator1,
> >> > >> > > > >>>>>> aggValueSerde1, storeName1);
> >> > >> > > > >>>>>> KTable<K, V2> table2 =
> >> > >> > > > >>>>>> builder.stream("topic2").groupByKey().aggregate(
> >> > initializer2
> >> > >> ,
> >> > >> > > > >>>>> aggregator2,
> >> > >> > > > >>>>>> aggValueSerde2, storeName2);
> >> > >> > > > >>>>>> KTable<K, V3> table3 =
> >> > >> > > > >>>>>> builder.stream("topic3").groupByKey().aggregate(
> >> > initializer3
> >> > >> ,
> >> > >> > > > >>>>> aggregator3,
> >> > >> > > > >>>>>> aggValueSerde3, storeName3);
> >> > >> > > > >>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> >> > >> > > > >>>>>> joinerOneAndTwo).outerJoin(table3,
> >> joinerOneTwoAndThree);
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> As you can see this creates 3 StateStores, requires 3
> >> > >> > > initializers,
> >> > >> > > > >>>> and 3
> >> > >> > > > >>>>>> aggValueSerdes. This also adds the pressure to user to
> >> > define
> >> > >> > what
> >> > >> > > > the
> >> > >> > > > >>>>>> intermediate values are going to be (V1, V2, V3). They
> >> are
> >> > >> left
> >> > >> > > > with a
> >> > >> > > > >>>>>> couple choices, first to make V1, V2, and V3 all the
> >> same
> >> > as
> >> > >> CG
> >> > >> > > and
> >> > >> > > > >> the
> >> > >> > > > >>>>> two
> >> > >> > > > >>>>>> joiners are more like mergers, or second make them
> >> > >> intermediate
> >> > >> > > > states
> >> > >> > > > >>>>> such
> >> > >> > > > >>>>>> as Topic1Map, Topic2Map, and Topic3Map and the joiners
> >> use
> >> > >> those
> >> > >> > > to
> >> > >> > > > >>>> build
> >> > >> > > > >>>>>> the final aggregate CG value. This is something the
> user
> >> > >> could
> >> > >> > > avoid
> >> > >> > > > >>>>>> thinking about with this KIP.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> When a new input arrives lets say at "topic1" it will
> >> first
> >> > >> go
> >> > >> > > > through
> >> > >> > > > >>>> a
> >> > >> > > > >>>>>> KStreamAggregate grabbing the current aggregate from
> >> > >> storeName1.
> >> > >> > > It
> >> > >> > > > >>>> will
> >> > >> > > > >>>>>> produce this in the form of the first intermediate
> value
> >> > and
> >> > >> get
> >> > >> > > > sent
> >> > >> > > > >>>>>> through a KTableKTableOuterJoin where it will look up
> >> the
> >> > >> > current
> >> > >> > > > >> value
> >> > >> > > > >>>>> of
> >> > >> > > > >>>>>> the key in storeName2. It will use the first joiner to
> >> > >> calculate
> >> > >> > > the
> >> > >> > > > >>>>> second
> >> > >> > > > >>>>>> intermediate value, which will go through an
> additional
> >> > >> > > > >>>>>> KTableKTableOuterJoin. Here it will look up the
> current
> >> > >> value of
> >> > >> > > the
> >> > >> > > > >>>> key
> >> > >> > > > >>>>> in
> >> > >> > > > >>>>>> storeName3 and use the second joiner to build the
> final
> >> > >> > aggregate
> >> > >> > > > >>>> value.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> If you think through all possibilities for incoming
> >> topics
> >> > >> you
> >> > >> > > will
> >> > >> > > > >> see
> >> > >> > > > >>>>>> that no matter which topic it comes in through all
> three
> >> > >> stores
> >> > >> > > are
> >> > >> > > > >>>>> queried
> >> > >> > > > >>>>>> and all of the joiners must get used.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> Topology wise for N incoming streams this creates N
> >> > >> > > > >>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins, and
> >> N-1
> >> > >> > > > >>>>>> KTableKTableJoinMergers.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> Example with Proposed API:
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> KGroupedStream<K, V1> grouped1 =
> >> builder.stream("topic1").
> >> > >> > > > >>>> groupByKey();
> >> > >> > > > >>>>>> KGroupedStream<K, V2> grouped2 =
> >> builder.stream("topic2").
> >> > >> > > > >>>> groupByKey();
> >> > >> > > > >>>>>> KGroupedStream<K, V3> grouped3 =
> >> builder.stream("topic3").
> >> > >> > > > >>>> groupByKey();
> >> > >> > > > >>>>>> KTable<K, CG> cogrouped =
> grouped1.cogroup(initializer1,
> >> > >> > > > aggregator1,
> >> > >> > > > >>>>>> aggValueSerde1, storeName1)
> >> > >> > > > >>>>>>       .cogroup(grouped2, aggregator2)
> >> > >> > > > >>>>>>       .cogroup(grouped3, aggregator3)
> >> > >> > > > >>>>>>       .aggregate();
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> As you can see this creates 1 StateStore, requires 1
> >> > >> > initializer,
> >> > >> > > > and
> >> > >> > > > >> 1
> >> > >> > > > >>>>>> aggValueSerde. The user no longer has to worry about
> the
> >> > >> > > > intermediate
> >> > >> > > > >>>>>> values and the joiners. All they have to think about
> is
> >> how
> >> > >> each
> >> > >> > > > >> stream
> >> > >> > > > >>>>>> impacts the creation of the final CG object.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> When a new input arrives lets say at "topic1" it will
> >> first
> >> > >> go
> >> > >> > > > through
> >> > >> > > > >>>> a
> >> > >> > > > >>>>>> KStreamAggreagte and grab the current aggregate from
> >> > >> storeName1.
> >> > >> > > It
> >> > >> > > > >>>> will
> >> > >> > > > >>>>>> add its incoming object to the aggregate, update the
> >> store
> >> > >> and
> >> > >> > > pass
> >> > >> > > > >> the
> >> > >> > > > >>>>> new
> >> > >> > > > >>>>>> aggregate on. This new aggregate goes through the
> >> > >> KStreamCogroup
> >> > >> > > > which
> >> > >> > > > >>>> is
> >> > >> > > > >>>>>> pretty much just a pass through processor and you are
> >> done.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> Topology wise for N incoming streams the new api will
> >> only
> >> > >> every
> >> > >> > > > >>>> create N
> >> > >> > > > >>>>>> KStreamAggregates and 1 KStreamCogroup.
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> >> > >> > > > >> matthias@confluent.io
> >> > >> > > > >>>>>
> >> > >> > > > >>>>>> wrote:
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>>> Kyle,
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
> >> but I
> >> > >> could
> >> > >> > > not
> >> > >> > > > >>>>>>> follow completely. Could you maybe add a more
> concrete
> >> > >> example,
> >> > >> > > > like
> >> > >> > > > >>>> 3
> >> > >> > > > >>>>>>> streams with 3 records each (plus expected result),
> and
> >> > show
> >> > >> > the
> >> > >> > > > >>>>>>> difference between current way to to implement it and
> >> the
> >> > >> > > proposed
> >> > >> > > > >>>> API?
> >> > >> > > > >>>>>>> This could also cover the internal processing to see
> >> what
> >> > >> store
> >> > >> > > > calls
> >> > >> > > > >>>>>>> would be required for both approaches etc.
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>> I think, it's pretty advanced stuff you propose, and
> it
> >> > >> would
> >> > >> > > help
> >> > >> > > > to
> >> > >> > > > >>>>>>> understand it better.
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>> Thanks a lot!
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>> -Matthias
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> >> > >> > > > >>>>>>>> I have made a pull request. It can be found here.
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> https://github.com/apache/kafka/pull/2975
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> I plan to write some more unit tests for my classes
> >> and
> >> > get
> >> > >> > > around
> >> > >> > > > >>>> to
> >> > >> > > > >>>>>>>> writing documentation for the public api additions.
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> One thing I was curious about is during the
> >> > >> > > > >>>>>>> KCogroupedStreamImpl#aggregate
> >> > >> > > > >>>>>>>> method I pass null to the KGroupedStream#
> >> > >> > repartitionIfRequired
> >> > >> > > > >>>>> method.
> >> > >> > > > >>>>>> I
> >> > >> > > > >>>>>>>> can't supply the store name because if more than one
> >> > >> grouped
> >> > >> > > > stream
> >> > >> > > > >>>>>>>> repartitions an error is thrown. Is there some name
> >> that
> >> > >> > someone
> >> > >> > > > >>>> can
> >> > >> > > > >>>>>>>> recommend or should I leave the null and allow it to
> >> fall
> >> > >> back
> >> > >> > > to
> >> > >> > > > >>>> the
> >> > >> > > > >>>>>>>> KGroupedStream.name?
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> Should this be expanded to handle grouped tables?
> This
> >> > >> would
> >> > >> > be
> >> > >> > > > >>>>> pretty
> >> > >> > > > >>>>>>> easy
> >> > >> > > > >>>>>>>> for a normal aggregate but one allowing session
> stores
> >> > and
> >> > >> > > > windowed
> >> > >> > > > >>>>>>> stores
> >> > >> > > > >>>>>>>> would required KTableSessionWindowAggregate and
> >> > >> > > > >>>> KTableWindowAggregate
> >> > >> > > > >>>>>>>> implementations.
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> Thanks,
> >> > >> > > > >>>>>>>> Kyle
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> >> > >> > eno.thereska@gmail.com>
> >> > >> > > > >>>>> wrote:
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>>> I’ll look as well asap, sorry, been swamped.
> >> > >> > > > >>>>>>>>>
> >> > >> > > > >>>>>>>>> Eno
> >> > >> > > > >>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> >> > >> > damian.guy@gmail.com>
> >> > >> > > > >>>>> wrote:
> >> > >> > > > >>>>>>>>>>
> >> > >> > > > >>>>>>>>>> Hi Kyle,
> >> > >> > > > >>>>>>>>>>
> >> > >> > > > >>>>>>>>>> Thanks for the KIP. I apologize that i haven't had
> >> the
> >> > >> > chance
> >> > >> > > to
> >> > >> > > > >>>>> look
> >> > >> > > > >>>>>>> at
> >> > >> > > > >>>>>>>>>> the KIP yet, but will schedule some time to look
> >> into
> >> > it
> >> > >> > > > >>>> tomorrow.
> >> > >> > > > >>>>>> For
> >> > >> > > > >>>>>>>>> the
> >> > >> > > > >>>>>>>>>> implementation, can you raise a PR against kafka
> >> trunk
> >> > >> and
> >> > >> > > mark
> >> > >> > > > >>>> it
> >> > >> > > > >>>>> as
> >> > >> > > > >>>>>>>>> WIP?
> >> > >> > > > >>>>>>>>>> It will be easier to review what you have done.
> >> > >> > > > >>>>>>>>>>
> >> > >> > > > >>>>>>>>>> Thanks,
> >> > >> > > > >>>>>>>>>> Damian
> >> > >> > > > >>>>>>>>>>
> >> > >> > > > >>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> >> > >> > > > >>>>> winkelman.kyle@gmail.com
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>>>> wrote:
> >> > >> > > > >>>>>>>>>>
> >> > >> > > > >>>>>>>>>>> I am replying to this in hopes it will draw some
> >> > >> attention
> >> > >> > to
> >> > >> > > > my
> >> > >> > > > >>>>> KIP
> >> > >> > > > >>>>>>> as
> >> > >> > > > >>>>>>>>> I
> >> > >> > > > >>>>>>>>>>> haven't heard from anyone in a couple days. This
> >> is my
> >> > >> > first
> >> > >> > > > KIP
> >> > >> > > > >>>>> and
> >> > >> > > > >>>>>>> my
> >> > >> > > > >>>>>>>>>>> first large contribution to the project so I'm
> >> sure I
> >> > >> did
> >> > >> > > > >>>>> something
> >> > >> > > > >>>>>>>>> wrong.
> >> > >> > > > >>>>>>>>>>> ;)
> >> > >> > > > >>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> >> > >> > > > >>>>> winkelman.kyle@gmail.com>
> >> > >> > > > >>>>>>>>> wrote:
> >> > >> > > > >>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>> Hello all,
> >> > >> > > > >>>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>> I have created KIP-150 to facilitate discussion
> >> about
> >> > >> > adding
> >> > >> > > > >>>>>> cogroup
> >> > >> > > > >>>>>>> to
> >> > >> > > > >>>>>>>>>>>> the streams DSL.
> >> > >> > > > >>>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>> Please find the KIP here:
> >> > >> > > > >>>>>>>>>>>> https://cwiki.apache.org/
> >> > confluence/display/KAFKA/KIP-
> >> > >> > > > >>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> >> > >> > > > >>>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>> Please find my initial implementation here:
> >> > >> > > > >>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> >> > >> > > > >>>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>> Thanks,
> >> > >> > > > >>>>>>>>>>>> Kyle Winkelman
> >> > >> > > > >>>>>>>>>>>>
> >> > >> > > > >>>>>>>>>>>
> >> > >> > > > >>>>>>>>>
> >> > >> > > > >>>>>>>>>
> >> > >> > > > >>>>>>>>
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>>
> >> > >> > > > >>>>>>
> >> > >> > > > >>>>>
> >> > >> > > > >>>>
> >> > >> > > > >>
> >> > >> > > > >>
> >> > >> > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > -- Guozhang
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
> >
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
The KIP and PR have been updated please go take a look and vote.

For those worried about the [DISCUSS] Streams DSL/StateStore Refactoring
email thread affecting this I believe the cogroup methods fit well into the
streams dsl and won't need to change. We can update the aggregate methods
in the same way we choose to update them in KGroupedStream.

Thanks,
Kyle

On Jun 14, 2017 8:14 PM, "Bill Bejeck" <bb...@gmail.com> wrote:

> +1
>
> Thanks,
> Bill
>
> On Wed, Jun 14, 2017 at 8:10 PM, Xavier Léauté <xa...@confluent.io>
> wrote:
>
> > +1 from me
> >
> > any stream should be able to initialize the cogroup
> >
> > On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman <winkelman.kyle@gmail.com
> >
> > wrote:
> >
> > > I will update the kip to have only the aggregator in the first cogroup
> > call
> > > and the initializer and serde in the aggregate calls.
> > >
> > > This seems to be the consensus on the email chain.
> > >
> > > Thanks,
> > > Kyle
> > >
> > > On Jun 14, 2017 5:41 PM, wrote:
> > >
> > > That is not the case. No matter which stream the record comes in on the
> > > initializer is called if there is not currently an object in the store.
> > >
> > > On Jun 14, 2017 5:12 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> > >
> > > While regarding where we should ask users to set serdes: I think I'm
> > > convinced by Xavier that they should be in the `aggregate` call (but
> only
> > > those does not pass in a state store supplier) instead of the
> > > `KStream#cogroup` call to be consistent with other aggregate functions.
> > >
> > > BTW another motivation for me to suggest keeping the initializer on the
> > > first stream is that by reviewing the PR (some time ago though, so
> again
> > I
> > > might be wrong) we will trigger the initializer only when we received
> an
> > > incoming record from the first stream whose key is not in the state
> store
> > > yet, while for other streams we will just drop it on the floor. If that
> > is
> > > actually not the case, that we call initializer on any one of the
> > > co-grouped streams' incoming records, then I'm open to set the
> > initializer
> > > at the `aggregate` call as well.
> > >
> > >
> > > Guozhang
> > >
> > > On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > I'd suggest we do not block this KIP until the serde work has been
> > sorted
> > > > out: we cannot estimate yet how long it will take yet. Instead let's
> > say
> > > > make an agreement on where we want to specify the serdes: whether on
> > the
> > > > first co-group call or on the aggregate call.
> > > >
> > > > Also about the initializer specification I actually felt that the
> first
> > > > cogrouped stream is special (Kyle please feel free to correct me if
> I'm
> > > > wrong) and that is why I thought it is better to specify the
> > initializer
> > > at
> > > > the beginning: since from the typing you can see that the final
> > > aggregated
> > > > value type is defined to be the same as the first co-grouped stream,
> > and
> > > > for any intermediate stream to co-group, their value types not be
> > > inherited
> > > > but the value be "incorporated" into the original stream:
> > > >
> > > >  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> > > > groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
> > > >
> > > > Note that we do not have a cogroup function that returns
> > > > CogroupedKStream<K, T>.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com>
> > wrote:
> > > >
> > > >> +1 on deferring discussion on Serdes until API improvements are
> ironed
> > > >> out.
> > > >>
> > > >> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <
> > matthias@confluent.io
> > > >
> > > >> wrote:
> > > >>
> > > >> > Hi,
> > > >> >
> > > >> > I am just catching up on this thread. (1) as most people agree, we
> > > >> > should not add anything to KStreamBuilder (btw: we actually plan
> to
> > > move
> > > >> > #merge() to KStream and deprecate it on KStreamBuilder as it's a
> > quite
> > > >> > unnatural API atm).
> > > >> >
> > > >> > About specifying Serdes: there is still the idea to improve to
> > overall
> > > >> > API from the current "we are adding more overloads"-pattern to a
> > > >> > builder-like pattern. This might make the whole discussion void if
> > we
> > > do
> > > >> > this. Thus, it might make sense to keep this in mind (or even
> delay
> > > this
> > > >> > KIP?). It seems a waste of time to discuss all this if we are
> going
> > to
> > > >> > chance it in 2 month anyway... Just saying.
> > > >> >
> > > >> >
> > > >> > -Matthias
> > > >> >
> > > >> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> > > >> > > You're right, I haven't thought of that.
> > > >> > >
> > > >> > > Cheers,
> > > >> > >
> > > >> > > Michał
> > > >> > >
> > > >> > >
> > > >> > > On 13/06/17 13:00, Kyle Winkelman wrote:
> > > >> > >> First, I would prefer not calling it aggregate because there
> are
> > > >> already
> > > >> > >> plenty of aggregate methods.
> > > >> > >>
> > > >> > >> Second, I dont think this would really work because after each
> > > >> aggregate
> > > >> > >> you now have a unique KTable (someone may want a table with 4
> > > streams
> > > >> > and
> > > >> > >> reuse those 4 in another table but with one more stream added)
> > and
> > > >> > unless
> > > >> > >> we completely duplicate everything every time this isnt really
> > > >> possible.
> > > >> > >> Additionally, the cogroup way just requires 1 more call to
> create
> > > two
> > > >> > >> different tables (normal, windowed, and session windowed) this
> > new
> > > >> way
> > > >> > >> would require copying the aggregate chain.
> > > >> > >>
> > > >> > >> Another way to think about it is with cogroup we know that when
> > > they
> > > >> > call
> > > >> > >> aggregate they arent going to be adding any more aggregators to
> > > that
> > > >> > table
> > > >> > >> but your way requires us to assume they are done adding
> > aggregators
> > > >> > after
> > > >> > >> each call so we must return a ktable just to possibly not need
> to
> > > >> have
> > > >> > >> created it.
> > > >> > >>
> > > >> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> > > >> > michal.borowiecki@openbet.com>
> > > >> > >> wrote:
> > > >> > >>
> > > >> > >>> Actually, just had a thought. It started with naming.
> > > >> > >>>
> > > >> > >>> Are we actually co-grouping these streams or are we
> > co-aggregating
> > > >> > them?
> > > >> > >>>
> > > >> > >>> After all, in each of the cogroup calls we are providing an
> > > >> Aggregator
> > > >> > >>> implementation.
> > > >> > >>>
> > > >> > >>>
> > > >> > >>> If they are really co-aggregated, why don't we turn this
> around:
> > > >> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> > > >> > groupByKey();
> > > >> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> > > >> > groupByKey();
> > > >> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> > > >> > groupByKey();
> > > >> > >>>
> > > >> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1,
> > > aggregator1,
> > > >> > >>> aggValueSerde1) // this is the unchanged aggregate method
> > > >> > >>>         .aggregate(grouped2, aggregator2)  // this is a new
> > method
> > > >> > >>>         .aggregate(grouped3, aggregator3); // ditto
> > > >> > >>>
> > > >> > >>> This means instead of adding cogroup methods on KGroupStream
> > > >> interface,
> > > >> > >>> adding aggregate method on KTable interface.
> > > >> > >>>
> > > >> > >>> Is that feasible?
> > > >> > >>>
> > > >> > >>> Cheers,
> > > >> > >>> Michał
> > > >> > >>>
> > > >> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> > > >> > >>>
> > > >> > >>> Also, I still feel that putting initializer on the first
> cogroup
> > > can
> > > >> > >>> mislead users into thinking the first stream is in some way
> > > special.
> > > >> > >>> Just my 5c.
> > > >> > >>> Michał
> > > >> > >>>
> > > >> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> > > >> > >>>
> > > >> > >>> Agree completely with the argument for serdes belonging in the
> > > same
> > > >> > place
> > > >> > >>> as the state store name, which is in the aggregate method.
> > > >> > >>>
> > > >> > >>> Cheers,
> > > >> > >>>
> > > >> > >>> Michał
> > > >> > >>>
> > > >> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> > > >> > >>>
> > > >> > >>> I think we are discussing two separate things here, so it
> might
> > be
> > > >> > worth
> > > >> > >>> clarifying:
> > > >> > >>>
> > > >> > >>> 1) the position of the initializer with respect to the
> > > aggregators.
> > > >> If
> > > >> > I
> > > >> > >>> understand correctly, Guozhang seems to think it is more
> natural
> > > to
> > > >> > specify
> > > >> > >>> the initializer first, despite it not bearing any relation to
> > the
> > > >> first
> > > >> > >>> aggregator. I can see the argument for specifying the
> > initializer
> > > >> > first,
> > > >> > >>> but I think it is debatable whether mixing it into the first
> > > cogroup
> > > >> > call
> > > >> > >>> leads to a cleaner API or not.
> > > >> > >>>
> > > >> > >>> 2) where the serde should be defined (if necessary). Looking
> at
> > > our
> > > >> > >>> existing APIs in KGroupedStreams, we always offer two
> > aggregate()
> > > >> > >>> methods. The first one takes the name of the store and
> > associated
> > > >> > aggregate
> > > >> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> > > >> initializer,
> > > >> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> > > >> > aggValueSerde,
> > > >> > >>> String queryableStoreName)
> > > >> > >>> The second one only takes a state store supplier, and does not
> > > >> specify
> > > >> > any
> > > >> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> > > >> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator,
> > > final
> > > >> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> > > >> > >>> Presumably, when specifying a state store supplier it
> shouldn't
> > be
> > > >> > >>> necessary to specify an aggregate value serde, since the
> > provided
> > > >> > >>> statestore might not need to serialize the values (e.g. it may
> > > just
> > > >> > keep
> > > >> > >>> them as regular objects in heap) or it may have its own
> > > >> > >>> internal serialization format.
> > > >> > >>>
> > > >> > >>> For consistency I think it would be valuable to preserve the
> > same
> > > >> two
> > > >> > >>> aggregate methods for cogroup as well. Since the serde is only
> > > >> > required in
> > > >> > >>> one of the two cases, I believe the serde has no place in the
> > > first
> > > >> > >>> cogroup() call and should only have to be specified as part of
> > the
> > > >> > >>> aggregate() method that takes a state store name. In the case
> > of a
> > > >> > state
> > > >> > >>> store supplier, no serde would be necessary.
> > > >> > >>>
> > > >> > >>>
> > > >> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > >> > wrote:
> > > >> > >>>
> > > >> > >>>> I'd agree that the aggregate value serde and the initializer
> > does
> > > >> not
> > > >> > >>>> bear direct relationship with the first `cogroup` calls, but
> > > after
> > > >> I
> > > >> > tried
> > > >> > >>>> to write some example code with these two different set of
> > APIs I
> > > >> > felt the
> > > >> > >>>> current APIs just program more naturally.
> > > >> > >>>>
> > > >> > >>>> I know it is kinda subjective, but I do think that user
> > > experience
> > > >> > may be
> > > >> > >>>> more important as a deciding factor than the logical argument
> > for
> > > >> > public
> > > >> > >>>> interfaces. So I'd recommend people to also try out writing
> > some
> > > >> > example
> > > >> > >>>> lines also and we can circle back and discuss which one feels
> > > more
> > > >> > natural
> > > >> > >>>> to write code.
> > > >> > >>>>
> > > >> > >>>>
> > > >> > >>>> Guozhang
> > > >> > >>>>
> > > >> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> > > >> > >>>> michal.borowiecki@openbet.com> wrote:
> > > >> > >>>>
> > > >> > >>>>> I feel it would make more sense to move the initializer and
> > > serde
> > > >> to
> > > >> > the
> > > >> > >>>>> final aggregate statement, since the serde only applies to
> the
> > > >> state
> > > >> > >>>>> store,
> > > >> > >>>>> and the initializer doesn't bear any relation to the first
> > group
> > > >> in
> > > >> > >>>>> particular.
> > > >> > >>>>>
> > > >> > >>>>> +1 for moving initializer and serde from cogroup() to the
> > > >> aggregate()
> > > >> > >>>>> for the reasons mentioned above.
> > > >> > >>>>>
> > > >> > >>>>> Cheers,
> > > >> > >>>>>
> > > >> > >>>>> Michał
> > > >> > >>>>>
> > > >> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> > > >> > >>>>>
> > > >> > >>>> Note that although the internal `AbstractStoreSupplier` does
> > > >> maintain
> > > >> > the
> > > >> > >>>>> key-value serdes, we do not enforce the interface of
> > > >> > `StateStoreSupplier`
> > > >> > >>>>> to always retain that information, and hence we cannot
> assume
> > > that
> > > >> > >>>>> StateStoreSuppliers always retain key / value serdes.
> > > >> > >>>>>
> > > >> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> > > >> xavier@confluent.io>
> > > >> > <xa...@confluent.io> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Another reason for the serde not to be in the first cogroup
> > > call,
> > > >> is
> > > >> > that
> > > >> > >>>>> the serde should not be required if you pass a
> > > StateStoreSupplier
> > > >> to
> > > >> > >>>>> aggregate()
> > > >> > >>>>>
> > > >> > >>>>> Regarding the aggregated type <T> I don't the why
> initializer
> > > >> should
> > > >> > be
> > > >> > >>>>> favored over aggregator to define the type. In my mind
> > > separating
> > > >> the
> > > >> > >>>>> initializer into the last aggregate call clearly indicates
> > that
> > > >> the
> > > >> > >>>>> initializer is independent of any of the aggregators or
> > streams
> > > >> and
> > > >> > that we
> > > >> > >>>>> don't wait for grouped1 events to initialize the co-group.
> > > >> > >>>>>
> > > >> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <
> > > wangguoz@gmail.com
> > > >
> > > >> <
> > > >> > wangguoz@gmail.com> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> On a second thought... This is the current proposal API
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> ```
> > > >> > >>>>>
> > > >> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> > > >> initializer,
> > > >> > >>>>>
> > > >> > >>>>> final
> > > >> > >>>>>
> > > >> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final
> Serde<T>
> > > >> > >>>>> aggValueSerde)
> > > >> > >>>>>
> > > >> > >>>>> ```
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> If we do not have the initializer in the first co-group it
> > might
> > > >> be
> > > >> > a bit
> > > >> > >>>>> awkward for users to specify the aggregator that returns a
> > typed
> > > >> <T>
> > > >> > >>>>>
> > > >> > >>>>> value?
> > > >> > >>>>>
> > > >> > >>>>> Maybe it is still better to put these two functions in the
> > same
> > > >> api?
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Guozhang
> > > >> > >>>>>
> > > >> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
> > > >> wangguoz@gmail.com>
> > > >> > <wa...@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> This suggestion lgtm. I would vote for the first alternative
> > > than
> > > >> > >>>>>
> > > >> > >>>>> adding
> > > >> > >>>>>
> > > >> > >>>>> it to the `KStreamBuilder` though.
> > > >> > >>>>>
> > > >> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> > > >> xavier@confluent.io>
> > > >> > <xa...@confluent.io>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> I have a minor suggestion to make the API a little bit more
> > > >> > symmetric.
> > > >> > >>>>> I feel it would make more sense to move the initializer and
> > > serde
> > > >> to
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> final aggregate statement, since the serde only applies to
> the
> > > >> state
> > > >> > >>>>> store,
> > > >> > >>>>> and the initializer doesn't bear any relation to the first
> > group
> > > >> in
> > > >> > >>>>> particular. It would end up looking like this:
> > > >> > >>>>>
> > > >> > >>>>> KTable<K, CG> cogrouped =
> > > >> > >>>>>     grouped1.cogroup(aggregator1)
> > > >> > >>>>>             .cogroup(grouped2, aggregator2)
> > > >> > >>>>>             .cogroup(grouped3, aggregator3)
> > > >> > >>>>>             .aggregate(initializer1, aggValueSerde,
> > storeName1);
> > > >> > >>>>>
> > > >> > >>>>> Alternatively, we could move the the first cogroup() method
> to
> > > >> > >>>>> KStreamBuilder, similar to how we have .merge()
> > > >> > >>>>> and end up with an api that would be even more symmetric.
> > > >> > >>>>>
> > > >> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> > > >> > >>>>>               .cogroup(grouped2, aggregator2)
> > > >> > >>>>>               .cogroup(grouped3, aggregator3)
> > > >> > >>>>>               .aggregate(initializer1, aggValueSerde,
> > > storeName1);
> > > >> > >>>>>
> > > >> > >>>>> This doesn't have to be a blocker, but I thought it would
> make
> > > the
> > > >> > API
> > > >> > >>>>> just
> > > >> > >>>>> a tad cleaner.
> > > >> > >>>>>
> > > >> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <
> > > wangguoz@gmail.com>
> > > >> <
> > > >> > wangguoz@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> Kyle,
> > > >> > >>>>>
> > > >> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Guozhang
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <
> > jim@jagunet.com>
> > > <
> > > >> > jim@jagunet.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> This makes much more sense to me. +1
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > > >> > >>>>>
> > > >> > >>>>> winkelman.kyle@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> I have updated the KIP and my PR. Let me know what you
> think.
> > > >> > >>>>> To created a cogrouped stream just call cogroup on a
> > > >> > >>>>>
> > > >> > >>>>> KgroupedStream
> > > >> > >>>>>
> > > >> > >>>>> and
> > > >> > >>>>>
> > > >> > >>>>> supply the initializer, aggValueSerde, and an aggregator.
> Then
> > > >> > >>>>>
> > > >> > >>>>> continue
> > > >> > >>>>>
> > > >> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> > > >> > >>>>>
> > > >> > >>>>> many
> > > >> > >>>>>
> > > >> > >>>>> aggregate
> > > >> > >>>>>
> > > >> > >>>>> calls to create a KTable.
> > > >> > >>>>>
> > > >> > >>>>> Thanks,
> > > >> > >>>>> Kyle
> > > >> > >>>>>
> > > >> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com>
> <
> > > >> > damian.guy@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> Hi Kyle,
> > > >> > >>>>>
> > > >> > >>>>> Thanks for the update. I think just one initializer makes
> > sense
> > > >> > >>>>>
> > > >> > >>>>> as
> > > >> > >>>>>
> > > >> > >>>>> it
> > > >> > >>>>>
> > > >> > >>>>> should only be called once per key and generally it is just
> > > >> > >>>>>
> > > >> > >>>>> going
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> create
> > > >> > >>>>>
> > > >> > >>>>> a new instance of whatever the Aggregate class is.
> > > >> > >>>>>
> > > >> > >>>>> Cheers,
> > > >> > >>>>> Damian
> > > >> > >>>>>
> > > >> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > > >> > >>>>>
> > > >> > >>>>> winkelman.kyle@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Hello all,
> > > >> > >>>>>
> > > >> > >>>>> I have spent some more time on this and the best
> alternative I
> > > >> > >>>>>
> > > >> > >>>>> have
> > > >> > >>>>>
> > > >> > >>>>> come
> > > >> > >>>>>
> > > >> > >>>>> up
> > > >> > >>>>>
> > > >> > >>>>> with is:
> > > >> > >>>>> KGroupedStream has a single cogroup call that takes an
> > > >> > >>>>>
> > > >> > >>>>> initializer
> > > >> > >>>>>
> > > >> > >>>>> and
> > > >> > >>>>>
> > > >> > >>>>> an
> > > >> > >>>>>
> > > >> > >>>>> aggregator.
> > > >> > >>>>> CogroupedKStream has a cogroup call that takes additional
> > > >> > >>>>>
> > > >> > >>>>> groupedStream
> > > >> > >>>>>
> > > >> > >>>>> aggregator pairs.
> > > >> > >>>>> CogroupedKStream has multiple aggregate methods that create
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> different
> > > >> > >>>>>
> > > >> > >>>>> stores.
> > > >> > >>>>>
> > > >> > >>>>> I plan on updating the kip but I want people's input on if
> we
> > > >> > >>>>>
> > > >> > >>>>> should
> > > >> > >>>>>
> > > >> > >>>>> have
> > > >> > >>>>>
> > > >> > >>>>> the initializer be passed in once at the beginning or if we
> > > >> > >>>>>
> > > >> > >>>>> should
> > > >> > >>>>>
> > > >> > >>>>> instead
> > > >> > >>>>>
> > > >> > >>>>> have the initializer be required for each call to one of the
> > > >> > >>>>>
> > > >> > >>>>> aggregate
> > > >> > >>>>>
> > > >> > >>>>> calls. The first makes more sense to me but doesnt allow the
> > > >> > >>>>>
> > > >> > >>>>> user
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> specify different initializers for different tables.
> > > >> > >>>>>
> > > >> > >>>>> Thanks,
> > > >> > >>>>> Kyle
> > > >> > >>>>>
> > > >> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > > >> > >>>>>
> > > >> > >>>>> winkelman.kyle@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Yea I really like that idea I'll see what I can do to update
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> kip
> > > >> > >>>>>
> > > >> > >>>>> and
> > > >> > >>>>>
> > > >> > >>>>> my pr when I have some time. I'm not sure how well creating
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> kstreamaggregates will go though because at that point I
> will
> > > >> > >>>>>
> > > >> > >>>>> have
> > > >> > >>>>>
> > > >> > >>>>> thrown
> > > >> > >>>>>
> > > >> > >>>>> away the type of the values. It will be type safe I just may
> > > >> > >>>>>
> > > >> > >>>>> need to
> > > >> > >>>>>
> > > >> > >>>>> do a
> > > >> > >>>>>
> > > >> > >>>>> little forcing.
> > > >> > >>>>>
> > > >> > >>>>> Thanks,
> > > >> > >>>>> Kyle
> > > >> > >>>>>
> > > >> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <
> wangguoz@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> Kyle,
> > > >> > >>>>>
> > > >> > >>>>> Thanks for the explanations, my previous read on the wiki
> > > >> > >>>>>
> > > >> > >>>>> examples
> > > >> > >>>>>
> > > >> > >>>>> was
> > > >> > >>>>>
> > > >> > >>>>> wrong.
> > > >> > >>>>>
> > > >> > >>>>> So I guess my motivation should be "reduced" to: can we move
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> window
> > > >> > >>>>>
> > > >> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > > >> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > > >> > >>>>>
> > > >> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > > >> > >>>>>
> > > >> > >>>>> from
> > > >> > >>>>>
> > > >> > >>>>> 3
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> 2.
> > > >> > >>>>>
> > > >> > >>>>> 2. major: this is for extensibility of the APIs, and since
> > > >> > >>>>>
> > > >> > >>>>> we
> > > >> > >>>>>
> > > >> > >>>>> are
> > > >> > >>>>>
> > > >> > >>>>> removing
> > > >> > >>>>>
> > > >> > >>>>> the "Evolving" annotations on Streams it may be harder to
> > > >> > >>>>>
> > > >> > >>>>> change it
> > > >> > >>>>>
> > > >> > >>>>> again
> > > >> > >>>>>
> > > >> > >>>>> in the future. The extended use cases are that people wanted
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> have
> > > >> > >>>>>
> > > >> > >>>>> windowed running aggregates on different granularities, e.g.
> > > >> > >>>>>
> > > >> > >>>>> "give
> > > >> > >>>>>
> > > >> > >>>>> me
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> counts per-minute, per-hour, per-day and per-week", and
> > > >> > >>>>>
> > > >> > >>>>> today
> > > >> > >>>>>
> > > >> > >>>>> in
> > > >> > >>>>>
> > > >> > >>>>> DSL
> > > >> > >>>>>
> > > >> > >>>>> we
> > > >> > >>>>>
> > > >> > >>>>> need to specify that case in multiple aggregate operators,
> > > >> > >>>>>
> > > >> > >>>>> which
> > > >> > >>>>>
> > > >> > >>>>> gets
> > > >> > >>>>>
> > > >> > >>>>> a
> > > >> > >>>>>
> > > >> > >>>>> state store / changelog, etc. And it is possible to optimize
> > > >> > >>>>>
> > > >> > >>>>> it
> > > >> > >>>>>
> > > >> > >>>>> as
> > > >> > >>>>>
> > > >> > >>>>> well
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> a single state store. Its implementation would be tricky as
> > > >> > >>>>>
> > > >> > >>>>> you
> > > >> > >>>>>
> > > >> > >>>>> need
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> contain different lengthed windows within your window store
> > > >> > >>>>>
> > > >> > >>>>> but
> > > >> > >>>>>
> > > >> > >>>>> just
> > > >> > >>>>>
> > > >> > >>>>> from
> > > >> > >>>>>
> > > >> > >>>>> the public API point of view, it could be specified as:
> > > >> > >>>>>
> > > >> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > > >> > >>>>> "state-store-name");
> > > >> > >>>>>
> > > >> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > > >> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > > >> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > > >> > >>>>>
> > > >> > >>>>> while underlying we are only using a single store
> > > >> > >>>>>
> > > >> > >>>>> "state-store-name"
> > > >> > >>>>>
> > > >> > >>>>> for
> > > >> > >>>>>
> > > >> > >>>>> it.
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Although this feature is out of the scope of this KIP, I'd
> > > >> > >>>>>
> > > >> > >>>>> like
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> discuss
> > > >> > >>>>>
> > > >> > >>>>> if we can "leave the door open" to make such changes without
> > > >> > >>>>>
> > > >> > >>>>> modifying
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> public APIs .
> > > >> > >>>>>
> > > >> > >>>>> Guozhang
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > > >> > >>>>>
> > > >> > >>>>> winkelman.kyle@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> I allow defining a single window/sessionwindow one time
> > > >> > >>>>>
> > > >> > >>>>> when
> > > >> > >>>>>
> > > >> > >>>>> you
> > > >> > >>>>>
> > > >> > >>>>> make
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> cogroup call from a KGroupedStream. From then on you are
> > > >> > >>>>>
> > > >> > >>>>> using
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> cogroup
> > > >> > >>>>>
> > > >> > >>>>> call from with in CogroupedKStream which doesnt accept any
> > > >> > >>>>>
> > > >> > >>>>> additional
> > > >> > >>>>>
> > > >> > >>>>> windows/sessionwindows.
> > > >> > >>>>>
> > > >> > >>>>> Is this what you meant by your question or did I
> > > >> > >>>>>
> > > >> > >>>>> misunderstand?
> > > >> > >>>>>
> > > >> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> > > >> > >>>>>
> > > >> > >>>>> wangguoz@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> Another question that came to me is on "window alignment":
> > > >> > >>>>>
> > > >> > >>>>> from
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> KIP
> > > >> > >>>>>
> > > >> > >>>>> it
> > > >> > >>>>>
> > > >> > >>>>> seems you are allowing users to specify a (potentially
> > > >> > >>>>>
> > > >> > >>>>> different)
> > > >> > >>>>>
> > > >> > >>>>> window
> > > >> > >>>>>
> > > >> > >>>>> spec in each co-grouped input stream. So if these window
> > > >> > >>>>>
> > > >> > >>>>> specs
> > > >> > >>>>>
> > > >> > >>>>> are
> > > >> > >>>>>
> > > >> > >>>>> different how should we "align" them with different input
> > > >> > >>>>>
> > > >> > >>>>> streams? I
> > > >> > >>>>>
> > > >> > >>>>> think
> > > >> > >>>>>
> > > >> > >>>>> it is more natural to only specify on window spec in the
> > > >> > >>>>>
> > > >> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> And remove it from the cogroup() functions. WDYT?
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> Guozhang
> > > >> > >>>>>
> > > >> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > > >> > >>>>>
> > > >> > >>>>> wangguoz@gmail.com>
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> Thanks for the proposal Kyle, this is a quite common use
> > > >> > >>>>>
> > > >> > >>>>> case
> > > >> > >>>>>
> > > >> > >>>>> to
> > > >> > >>>>>
> > > >> > >>>>> support
> > > >> > >>>>>
> > > >> > >>>>> such multi-way table join (i.e. N source tables with N
> > > >> > >>>>>
> > > >> > >>>>> aggregate
> > > >> > >>>>>
> > > >> > >>>>> func)
> > > >> > >>>>>
> > > >> > >>>>> with
> > > >> > >>>>>
> > > >> > >>>>> a single store and N+1 serdes, I have seen lots of people
> > > >> > >>>>>
> > > >> > >>>>> using
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> low-level PAPI to achieve this goal.
> > > >> > >>>>>
> > > >> > >>>>>
> > > >> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > > >> > >>>>>
> > > >> > >>>>> winkelman.kyle@gmail.com
> > > >> > >>>>>
> > > >> > >>>>> wrote:
> > > >> > >>>>>
> > > >> > >>>>> I like your point about not handling other cases such as
> > > >> > >>>>>
> > > >> > >>>>> count
> > > >> > >>>>>
> > > >> > >>>>> and
> > > >> > >>>>>
> > > >> > >>>>> reduce.
> > > >> > >>>>>
> > > >> > >>>>> I think that reduce may not make sense because reduce
> > > >> > >>>>>
> > > >> > >>>>> assumes
> > > >> > >>>>>
> > > >> > >>>>> that
> > > >> > >>>>>
> > > >> > >>>>> the
> > > >> > >>>>>
> > > >> > >>>>> input values are the same as the output values. With
> > > >> > >>>>>
> > > >> > >>>>> cogroup
> > > >> > >>>>>
> > > >> > >>>>> ...
> > > >> > >
> > > >> > > --
> > > >> > > Signature
> > > >> > > <http://www.openbet.com/>     Michal Borowiecki
> > > >> > > Senior Software Engineer L4
> > > >> > >       T:      +44 208 742 1600 <+44%2020%208742%201600>
> > > >> > >
> > > >> > >
> > > >> > >       +44 203 249 8448 <+44%2020%203249%208448>
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > >       E:      michal.borowiecki@openbet.com
> > > >> > >       W:      www.openbet.com <http://www.openbet.com/>
> > > >> > >
> > > >> > >
> > > >> > >       OpenBet Ltd
> > > >> > >
> > > >> > >       Chiswick Park Building 9
> > > >> > >
> > > >> > >       566 Chiswick High Rd
> > > >> > >
> > > >> > >       London
> > > >> > >
> > > >> > >       W4 5XT
> > > >> > >
> > > >> > >       UK
> > > >> > >
> > > >> > >
> > > >> > > <https://www.openbet.com/email_promo>
> > > >> > >
> > > >> > > This message is confidential and intended only for the
> addressee.
> > If
> > > >> you
> > > >> > > have received this message in error, please immediately notify
> the
> > > >> > > postmaster@openbet.com <ma...@openbet.com> and
> delete
> > > it
> > > >> > > from your system as well as any copies. The content of e-mails
> as
> > > well
> > > >> > > as traffic data may be monitored by OpenBet for employment and
> > > >> security
> > > >> > > purposes. To protect the environment please do not print this
> > e-mail
> > > >> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> > > >> Building
> > > >> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
> > company
> > > >> > > registered in England and Wales. Registered no. 3134634. VAT no.
> > > >> > > GB927523612
> > > >> > >
> > > >> >
> > > >> >
> > > >>
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Bill Bejeck <bb...@gmail.com>.
+1

Thanks,
Bill

On Wed, Jun 14, 2017 at 8:10 PM, Xavier Léauté <xa...@confluent.io> wrote:

> +1 from me
>
> any stream should be able to initialize the cogroup
>
> On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman <wi...@gmail.com>
> wrote:
>
> > I will update the kip to have only the aggregator in the first cogroup
> call
> > and the initializer and serde in the aggregate calls.
> >
> > This seems to be the consensus on the email chain.
> >
> > Thanks,
> > Kyle
> >
> > On Jun 14, 2017 5:41 PM, wrote:
> >
> > That is not the case. No matter which stream the record comes in on the
> > initializer is called if there is not currently an object in the store.
> >
> > On Jun 14, 2017 5:12 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> >
> > While regarding where we should ask users to set serdes: I think I'm
> > convinced by Xavier that they should be in the `aggregate` call (but only
> > those does not pass in a state store supplier) instead of the
> > `KStream#cogroup` call to be consistent with other aggregate functions.
> >
> > BTW another motivation for me to suggest keeping the initializer on the
> > first stream is that by reviewing the PR (some time ago though, so again
> I
> > might be wrong) we will trigger the initializer only when we received an
> > incoming record from the first stream whose key is not in the state store
> > yet, while for other streams we will just drop it on the floor. If that
> is
> > actually not the case, that we call initializer on any one of the
> > co-grouped streams' incoming records, then I'm open to set the
> initializer
> > at the `aggregate` call as well.
> >
> >
> > Guozhang
> >
> > On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > I'd suggest we do not block this KIP until the serde work has been
> sorted
> > > out: we cannot estimate yet how long it will take yet. Instead let's
> say
> > > make an agreement on where we want to specify the serdes: whether on
> the
> > > first co-group call or on the aggregate call.
> > >
> > > Also about the initializer specification I actually felt that the first
> > > cogrouped stream is special (Kyle please feel free to correct me if I'm
> > > wrong) and that is why I thought it is better to specify the
> initializer
> > at
> > > the beginning: since from the typing you can see that the final
> > aggregated
> > > value type is defined to be the same as the first co-grouped stream,
> and
> > > for any intermediate stream to co-group, their value types not be
> > inherited
> > > but the value be "incorporated" into the original stream:
> > >
> > >  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> > > groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
> > >
> > > Note that we do not have a cogroup function that returns
> > > CogroupedKStream<K, T>.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com>
> wrote:
> > >
> > >> +1 on deferring discussion on Serdes until API improvements are ironed
> > >> out.
> > >>
> > >> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <
> matthias@confluent.io
> > >
> > >> wrote:
> > >>
> > >> > Hi,
> > >> >
> > >> > I am just catching up on this thread. (1) as most people agree, we
> > >> > should not add anything to KStreamBuilder (btw: we actually plan to
> > move
> > >> > #merge() to KStream and deprecate it on KStreamBuilder as it's a
> quite
> > >> > unnatural API atm).
> > >> >
> > >> > About specifying Serdes: there is still the idea to improve to
> overall
> > >> > API from the current "we are adding more overloads"-pattern to a
> > >> > builder-like pattern. This might make the whole discussion void if
> we
> > do
> > >> > this. Thus, it might make sense to keep this in mind (or even delay
> > this
> > >> > KIP?). It seems a waste of time to discuss all this if we are going
> to
> > >> > chance it in 2 month anyway... Just saying.
> > >> >
> > >> >
> > >> > -Matthias
> > >> >
> > >> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> > >> > > You're right, I haven't thought of that.
> > >> > >
> > >> > > Cheers,
> > >> > >
> > >> > > Michał
> > >> > >
> > >> > >
> > >> > > On 13/06/17 13:00, Kyle Winkelman wrote:
> > >> > >> First, I would prefer not calling it aggregate because there are
> > >> already
> > >> > >> plenty of aggregate methods.
> > >> > >>
> > >> > >> Second, I dont think this would really work because after each
> > >> aggregate
> > >> > >> you now have a unique KTable (someone may want a table with 4
> > streams
> > >> > and
> > >> > >> reuse those 4 in another table but with one more stream added)
> and
> > >> > unless
> > >> > >> we completely duplicate everything every time this isnt really
> > >> possible.
> > >> > >> Additionally, the cogroup way just requires 1 more call to create
> > two
> > >> > >> different tables (normal, windowed, and session windowed) this
> new
> > >> way
> > >> > >> would require copying the aggregate chain.
> > >> > >>
> > >> > >> Another way to think about it is with cogroup we know that when
> > they
> > >> > call
> > >> > >> aggregate they arent going to be adding any more aggregators to
> > that
> > >> > table
> > >> > >> but your way requires us to assume they are done adding
> aggregators
> > >> > after
> > >> > >> each call so we must return a ktable just to possibly not need to
> > >> have
> > >> > >> created it.
> > >> > >>
> > >> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> > >> > michal.borowiecki@openbet.com>
> > >> > >> wrote:
> > >> > >>
> > >> > >>> Actually, just had a thought. It started with naming.
> > >> > >>>
> > >> > >>> Are we actually co-grouping these streams or are we
> co-aggregating
> > >> > them?
> > >> > >>>
> > >> > >>> After all, in each of the cogroup calls we are providing an
> > >> Aggregator
> > >> > >>> implementation.
> > >> > >>>
> > >> > >>>
> > >> > >>> If they are really co-aggregated, why don't we turn this around:
> > >> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> > >> > groupByKey();
> > >> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> > >> > groupByKey();
> > >> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> > >> > groupByKey();
> > >> > >>>
> > >> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1,
> > aggregator1,
> > >> > >>> aggValueSerde1) // this is the unchanged aggregate method
> > >> > >>>         .aggregate(grouped2, aggregator2)  // this is a new
> method
> > >> > >>>         .aggregate(grouped3, aggregator3); // ditto
> > >> > >>>
> > >> > >>> This means instead of adding cogroup methods on KGroupStream
> > >> interface,
> > >> > >>> adding aggregate method on KTable interface.
> > >> > >>>
> > >> > >>> Is that feasible?
> > >> > >>>
> > >> > >>> Cheers,
> > >> > >>> Michał
> > >> > >>>
> > >> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> > >> > >>>
> > >> > >>> Also, I still feel that putting initializer on the first cogroup
> > can
> > >> > >>> mislead users into thinking the first stream is in some way
> > special.
> > >> > >>> Just my 5c.
> > >> > >>> Michał
> > >> > >>>
> > >> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> > >> > >>>
> > >> > >>> Agree completely with the argument for serdes belonging in the
> > same
> > >> > place
> > >> > >>> as the state store name, which is in the aggregate method.
> > >> > >>>
> > >> > >>> Cheers,
> > >> > >>>
> > >> > >>> Michał
> > >> > >>>
> > >> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> > >> > >>>
> > >> > >>> I think we are discussing two separate things here, so it might
> be
> > >> > worth
> > >> > >>> clarifying:
> > >> > >>>
> > >> > >>> 1) the position of the initializer with respect to the
> > aggregators.
> > >> If
> > >> > I
> > >> > >>> understand correctly, Guozhang seems to think it is more natural
> > to
> > >> > specify
> > >> > >>> the initializer first, despite it not bearing any relation to
> the
> > >> first
> > >> > >>> aggregator. I can see the argument for specifying the
> initializer
> > >> > first,
> > >> > >>> but I think it is debatable whether mixing it into the first
> > cogroup
> > >> > call
> > >> > >>> leads to a cleaner API or not.
> > >> > >>>
> > >> > >>> 2) where the serde should be defined (if necessary). Looking at
> > our
> > >> > >>> existing APIs in KGroupedStreams, we always offer two
> aggregate()
> > >> > >>> methods. The first one takes the name of the store and
> associated
> > >> > aggregate
> > >> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> > >> initializer,
> > >> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> > >> > aggValueSerde,
> > >> > >>> String queryableStoreName)
> > >> > >>> The second one only takes a state store supplier, and does not
> > >> specify
> > >> > any
> > >> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> > >> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator,
> > final
> > >> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> > >> > >>> Presumably, when specifying a state store supplier it shouldn't
> be
> > >> > >>> necessary to specify an aggregate value serde, since the
> provided
> > >> > >>> statestore might not need to serialize the values (e.g. it may
> > just
> > >> > keep
> > >> > >>> them as regular objects in heap) or it may have its own
> > >> > >>> internal serialization format.
> > >> > >>>
> > >> > >>> For consistency I think it would be valuable to preserve the
> same
> > >> two
> > >> > >>> aggregate methods for cogroup as well. Since the serde is only
> > >> > required in
> > >> > >>> one of the two cases, I believe the serde has no place in the
> > first
> > >> > >>> cogroup() call and should only have to be specified as part of
> the
> > >> > >>> aggregate() method that takes a state store name. In the case
> of a
> > >> > state
> > >> > >>> store supplier, no serde would be necessary.
> > >> > >>>
> > >> > >>>
> > >> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <
> wangguoz@gmail.com
> > >
> > >> > wrote:
> > >> > >>>
> > >> > >>>> I'd agree that the aggregate value serde and the initializer
> does
> > >> not
> > >> > >>>> bear direct relationship with the first `cogroup` calls, but
> > after
> > >> I
> > >> > tried
> > >> > >>>> to write some example code with these two different set of
> APIs I
> > >> > felt the
> > >> > >>>> current APIs just program more naturally.
> > >> > >>>>
> > >> > >>>> I know it is kinda subjective, but I do think that user
> > experience
> > >> > may be
> > >> > >>>> more important as a deciding factor than the logical argument
> for
> > >> > public
> > >> > >>>> interfaces. So I'd recommend people to also try out writing
> some
> > >> > example
> > >> > >>>> lines also and we can circle back and discuss which one feels
> > more
> > >> > natural
> > >> > >>>> to write code.
> > >> > >>>>
> > >> > >>>>
> > >> > >>>> Guozhang
> > >> > >>>>
> > >> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> > >> > >>>> michal.borowiecki@openbet.com> wrote:
> > >> > >>>>
> > >> > >>>>> I feel it would make more sense to move the initializer and
> > serde
> > >> to
> > >> > the
> > >> > >>>>> final aggregate statement, since the serde only applies to the
> > >> state
> > >> > >>>>> store,
> > >> > >>>>> and the initializer doesn't bear any relation to the first
> group
> > >> in
> > >> > >>>>> particular.
> > >> > >>>>>
> > >> > >>>>> +1 for moving initializer and serde from cogroup() to the
> > >> aggregate()
> > >> > >>>>> for the reasons mentioned above.
> > >> > >>>>>
> > >> > >>>>> Cheers,
> > >> > >>>>>
> > >> > >>>>> Michał
> > >> > >>>>>
> > >> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> > >> > >>>>>
> > >> > >>>> Note that although the internal `AbstractStoreSupplier` does
> > >> maintain
> > >> > the
> > >> > >>>>> key-value serdes, we do not enforce the interface of
> > >> > `StateStoreSupplier`
> > >> > >>>>> to always retain that information, and hence we cannot assume
> > that
> > >> > >>>>> StateStoreSuppliers always retain key / value serdes.
> > >> > >>>>>
> > >> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> > >> xavier@confluent.io>
> > >> > <xa...@confluent.io> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Another reason for the serde not to be in the first cogroup
> > call,
> > >> is
> > >> > that
> > >> > >>>>> the serde should not be required if you pass a
> > StateStoreSupplier
> > >> to
> > >> > >>>>> aggregate()
> > >> > >>>>>
> > >> > >>>>> Regarding the aggregated type <T> I don't the why initializer
> > >> should
> > >> > be
> > >> > >>>>> favored over aggregator to define the type. In my mind
> > separating
> > >> the
> > >> > >>>>> initializer into the last aggregate call clearly indicates
> that
> > >> the
> > >> > >>>>> initializer is independent of any of the aggregators or
> streams
> > >> and
> > >> > that we
> > >> > >>>>> don't wait for grouped1 events to initialize the co-group.
> > >> > >>>>>
> > >> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <
> > wangguoz@gmail.com
> > >
> > >> <
> > >> > wangguoz@gmail.com> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> On a second thought... This is the current proposal API
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> ```
> > >> > >>>>>
> > >> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> > >> initializer,
> > >> > >>>>>
> > >> > >>>>> final
> > >> > >>>>>
> > >> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> > >> > >>>>> aggValueSerde)
> > >> > >>>>>
> > >> > >>>>> ```
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> If we do not have the initializer in the first co-group it
> might
> > >> be
> > >> > a bit
> > >> > >>>>> awkward for users to specify the aggregator that returns a
> typed
> > >> <T>
> > >> > >>>>>
> > >> > >>>>> value?
> > >> > >>>>>
> > >> > >>>>> Maybe it is still better to put these two functions in the
> same
> > >> api?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Guozhang
> > >> > >>>>>
> > >> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
> > >> wangguoz@gmail.com>
> > >> > <wa...@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> This suggestion lgtm. I would vote for the first alternative
> > than
> > >> > >>>>>
> > >> > >>>>> adding
> > >> > >>>>>
> > >> > >>>>> it to the `KStreamBuilder` though.
> > >> > >>>>>
> > >> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> > >> xavier@confluent.io>
> > >> > <xa...@confluent.io>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> I have a minor suggestion to make the API a little bit more
> > >> > symmetric.
> > >> > >>>>> I feel it would make more sense to move the initializer and
> > serde
> > >> to
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> final aggregate statement, since the serde only applies to the
> > >> state
> > >> > >>>>> store,
> > >> > >>>>> and the initializer doesn't bear any relation to the first
> group
> > >> in
> > >> > >>>>> particular. It would end up looking like this:
> > >> > >>>>>
> > >> > >>>>> KTable<K, CG> cogrouped =
> > >> > >>>>>     grouped1.cogroup(aggregator1)
> > >> > >>>>>             .cogroup(grouped2, aggregator2)
> > >> > >>>>>             .cogroup(grouped3, aggregator3)
> > >> > >>>>>             .aggregate(initializer1, aggValueSerde,
> storeName1);
> > >> > >>>>>
> > >> > >>>>> Alternatively, we could move the the first cogroup() method to
> > >> > >>>>> KStreamBuilder, similar to how we have .merge()
> > >> > >>>>> and end up with an api that would be even more symmetric.
> > >> > >>>>>
> > >> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> > >> > >>>>>               .cogroup(grouped2, aggregator2)
> > >> > >>>>>               .cogroup(grouped3, aggregator3)
> > >> > >>>>>               .aggregate(initializer1, aggValueSerde,
> > storeName1);
> > >> > >>>>>
> > >> > >>>>> This doesn't have to be a blocker, but I thought it would make
> > the
> > >> > API
> > >> > >>>>> just
> > >> > >>>>> a tad cleaner.
> > >> > >>>>>
> > >> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <
> > wangguoz@gmail.com>
> > >> <
> > >> > wangguoz@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> Kyle,
> > >> > >>>>>
> > >> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Guozhang
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <
> jim@jagunet.com>
> > <
> > >> > jim@jagunet.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> This makes much more sense to me. +1
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >> > >>>>>
> > >> > >>>>> winkelman.kyle@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> I have updated the KIP and my PR. Let me know what you think.
> > >> > >>>>> To created a cogrouped stream just call cogroup on a
> > >> > >>>>>
> > >> > >>>>> KgroupedStream
> > >> > >>>>>
> > >> > >>>>> and
> > >> > >>>>>
> > >> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> > >> > >>>>>
> > >> > >>>>> continue
> > >> > >>>>>
> > >> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> > >> > >>>>>
> > >> > >>>>> many
> > >> > >>>>>
> > >> > >>>>> aggregate
> > >> > >>>>>
> > >> > >>>>> calls to create a KTable.
> > >> > >>>>>
> > >> > >>>>> Thanks,
> > >> > >>>>> Kyle
> > >> > >>>>>
> > >> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> > >> > damian.guy@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> Hi Kyle,
> > >> > >>>>>
> > >> > >>>>> Thanks for the update. I think just one initializer makes
> sense
> > >> > >>>>>
> > >> > >>>>> as
> > >> > >>>>>
> > >> > >>>>> it
> > >> > >>>>>
> > >> > >>>>> should only be called once per key and generally it is just
> > >> > >>>>>
> > >> > >>>>> going
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> create
> > >> > >>>>>
> > >> > >>>>> a new instance of whatever the Aggregate class is.
> > >> > >>>>>
> > >> > >>>>> Cheers,
> > >> > >>>>> Damian
> > >> > >>>>>
> > >> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > >> > >>>>>
> > >> > >>>>> winkelman.kyle@gmail.com
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Hello all,
> > >> > >>>>>
> > >> > >>>>> I have spent some more time on this and the best alternative I
> > >> > >>>>>
> > >> > >>>>> have
> > >> > >>>>>
> > >> > >>>>> come
> > >> > >>>>>
> > >> > >>>>> up
> > >> > >>>>>
> > >> > >>>>> with is:
> > >> > >>>>> KGroupedStream has a single cogroup call that takes an
> > >> > >>>>>
> > >> > >>>>> initializer
> > >> > >>>>>
> > >> > >>>>> and
> > >> > >>>>>
> > >> > >>>>> an
> > >> > >>>>>
> > >> > >>>>> aggregator.
> > >> > >>>>> CogroupedKStream has a cogroup call that takes additional
> > >> > >>>>>
> > >> > >>>>> groupedStream
> > >> > >>>>>
> > >> > >>>>> aggregator pairs.
> > >> > >>>>> CogroupedKStream has multiple aggregate methods that create
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> different
> > >> > >>>>>
> > >> > >>>>> stores.
> > >> > >>>>>
> > >> > >>>>> I plan on updating the kip but I want people's input on if we
> > >> > >>>>>
> > >> > >>>>> should
> > >> > >>>>>
> > >> > >>>>> have
> > >> > >>>>>
> > >> > >>>>> the initializer be passed in once at the beginning or if we
> > >> > >>>>>
> > >> > >>>>> should
> > >> > >>>>>
> > >> > >>>>> instead
> > >> > >>>>>
> > >> > >>>>> have the initializer be required for each call to one of the
> > >> > >>>>>
> > >> > >>>>> aggregate
> > >> > >>>>>
> > >> > >>>>> calls. The first makes more sense to me but doesnt allow the
> > >> > >>>>>
> > >> > >>>>> user
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> specify different initializers for different tables.
> > >> > >>>>>
> > >> > >>>>> Thanks,
> > >> > >>>>> Kyle
> > >> > >>>>>
> > >> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > >> > >>>>>
> > >> > >>>>> winkelman.kyle@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Yea I really like that idea I'll see what I can do to update
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> kip
> > >> > >>>>>
> > >> > >>>>> and
> > >> > >>>>>
> > >> > >>>>> my pr when I have some time. I'm not sure how well creating
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> kstreamaggregates will go though because at that point I will
> > >> > >>>>>
> > >> > >>>>> have
> > >> > >>>>>
> > >> > >>>>> thrown
> > >> > >>>>>
> > >> > >>>>> away the type of the values. It will be type safe I just may
> > >> > >>>>>
> > >> > >>>>> need to
> > >> > >>>>>
> > >> > >>>>> do a
> > >> > >>>>>
> > >> > >>>>> little forcing.
> > >> > >>>>>
> > >> > >>>>> Thanks,
> > >> > >>>>> Kyle
> > >> > >>>>>
> > >> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> Kyle,
> > >> > >>>>>
> > >> > >>>>> Thanks for the explanations, my previous read on the wiki
> > >> > >>>>>
> > >> > >>>>> examples
> > >> > >>>>>
> > >> > >>>>> was
> > >> > >>>>>
> > >> > >>>>> wrong.
> > >> > >>>>>
> > >> > >>>>> So I guess my motivation should be "reduced" to: can we move
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> window
> > >> > >>>>>
> > >> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > >> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >> > >>>>>
> > >> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > >> > >>>>>
> > >> > >>>>> from
> > >> > >>>>>
> > >> > >>>>> 3
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> 2.
> > >> > >>>>>
> > >> > >>>>> 2. major: this is for extensibility of the APIs, and since
> > >> > >>>>>
> > >> > >>>>> we
> > >> > >>>>>
> > >> > >>>>> are
> > >> > >>>>>
> > >> > >>>>> removing
> > >> > >>>>>
> > >> > >>>>> the "Evolving" annotations on Streams it may be harder to
> > >> > >>>>>
> > >> > >>>>> change it
> > >> > >>>>>
> > >> > >>>>> again
> > >> > >>>>>
> > >> > >>>>> in the future. The extended use cases are that people wanted
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> have
> > >> > >>>>>
> > >> > >>>>> windowed running aggregates on different granularities, e.g.
> > >> > >>>>>
> > >> > >>>>> "give
> > >> > >>>>>
> > >> > >>>>> me
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> counts per-minute, per-hour, per-day and per-week", and
> > >> > >>>>>
> > >> > >>>>> today
> > >> > >>>>>
> > >> > >>>>> in
> > >> > >>>>>
> > >> > >>>>> DSL
> > >> > >>>>>
> > >> > >>>>> we
> > >> > >>>>>
> > >> > >>>>> need to specify that case in multiple aggregate operators,
> > >> > >>>>>
> > >> > >>>>> which
> > >> > >>>>>
> > >> > >>>>> gets
> > >> > >>>>>
> > >> > >>>>> a
> > >> > >>>>>
> > >> > >>>>> state store / changelog, etc. And it is possible to optimize
> > >> > >>>>>
> > >> > >>>>> it
> > >> > >>>>>
> > >> > >>>>> as
> > >> > >>>>>
> > >> > >>>>> well
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> a single state store. Its implementation would be tricky as
> > >> > >>>>>
> > >> > >>>>> you
> > >> > >>>>>
> > >> > >>>>> need
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> contain different lengthed windows within your window store
> > >> > >>>>>
> > >> > >>>>> but
> > >> > >>>>>
> > >> > >>>>> just
> > >> > >>>>>
> > >> > >>>>> from
> > >> > >>>>>
> > >> > >>>>> the public API point of view, it could be specified as:
> > >> > >>>>>
> > >> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >> > >>>>> "state-store-name");
> > >> > >>>>>
> > >> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > >> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > >> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > >> > >>>>>
> > >> > >>>>> while underlying we are only using a single store
> > >> > >>>>>
> > >> > >>>>> "state-store-name"
> > >> > >>>>>
> > >> > >>>>> for
> > >> > >>>>>
> > >> > >>>>> it.
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Although this feature is out of the scope of this KIP, I'd
> > >> > >>>>>
> > >> > >>>>> like
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> discuss
> > >> > >>>>>
> > >> > >>>>> if we can "leave the door open" to make such changes without
> > >> > >>>>>
> > >> > >>>>> modifying
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> public APIs .
> > >> > >>>>>
> > >> > >>>>> Guozhang
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > >> > >>>>>
> > >> > >>>>> winkelman.kyle@gmail.com
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> I allow defining a single window/sessionwindow one time
> > >> > >>>>>
> > >> > >>>>> when
> > >> > >>>>>
> > >> > >>>>> you
> > >> > >>>>>
> > >> > >>>>> make
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> cogroup call from a KGroupedStream. From then on you are
> > >> > >>>>>
> > >> > >>>>> using
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> cogroup
> > >> > >>>>>
> > >> > >>>>> call from with in CogroupedKStream which doesnt accept any
> > >> > >>>>>
> > >> > >>>>> additional
> > >> > >>>>>
> > >> > >>>>> windows/sessionwindows.
> > >> > >>>>>
> > >> > >>>>> Is this what you meant by your question or did I
> > >> > >>>>>
> > >> > >>>>> misunderstand?
> > >> > >>>>>
> > >> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> > >> > >>>>>
> > >> > >>>>> wangguoz@gmail.com
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> Another question that came to me is on "window alignment":
> > >> > >>>>>
> > >> > >>>>> from
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> KIP
> > >> > >>>>>
> > >> > >>>>> it
> > >> > >>>>>
> > >> > >>>>> seems you are allowing users to specify a (potentially
> > >> > >>>>>
> > >> > >>>>> different)
> > >> > >>>>>
> > >> > >>>>> window
> > >> > >>>>>
> > >> > >>>>> spec in each co-grouped input stream. So if these window
> > >> > >>>>>
> > >> > >>>>> specs
> > >> > >>>>>
> > >> > >>>>> are
> > >> > >>>>>
> > >> > >>>>> different how should we "align" them with different input
> > >> > >>>>>
> > >> > >>>>> streams? I
> > >> > >>>>>
> > >> > >>>>> think
> > >> > >>>>>
> > >> > >>>>> it is more natural to only specify on window spec in the
> > >> > >>>>>
> > >> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> And remove it from the cogroup() functions. WDYT?
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> Guozhang
> > >> > >>>>>
> > >> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > >> > >>>>>
> > >> > >>>>> wangguoz@gmail.com>
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> Thanks for the proposal Kyle, this is a quite common use
> > >> > >>>>>
> > >> > >>>>> case
> > >> > >>>>>
> > >> > >>>>> to
> > >> > >>>>>
> > >> > >>>>> support
> > >> > >>>>>
> > >> > >>>>> such multi-way table join (i.e. N source tables with N
> > >> > >>>>>
> > >> > >>>>> aggregate
> > >> > >>>>>
> > >> > >>>>> func)
> > >> > >>>>>
> > >> > >>>>> with
> > >> > >>>>>
> > >> > >>>>> a single store and N+1 serdes, I have seen lots of people
> > >> > >>>>>
> > >> > >>>>> using
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> low-level PAPI to achieve this goal.
> > >> > >>>>>
> > >> > >>>>>
> > >> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >> > >>>>>
> > >> > >>>>> winkelman.kyle@gmail.com
> > >> > >>>>>
> > >> > >>>>> wrote:
> > >> > >>>>>
> > >> > >>>>> I like your point about not handling other cases such as
> > >> > >>>>>
> > >> > >>>>> count
> > >> > >>>>>
> > >> > >>>>> and
> > >> > >>>>>
> > >> > >>>>> reduce.
> > >> > >>>>>
> > >> > >>>>> I think that reduce may not make sense because reduce
> > >> > >>>>>
> > >> > >>>>> assumes
> > >> > >>>>>
> > >> > >>>>> that
> > >> > >>>>>
> > >> > >>>>> the
> > >> > >>>>>
> > >> > >>>>> input values are the same as the output values. With
> > >> > >>>>>
> > >> > >>>>> cogroup
> > >> > >>>>>
> > >> > >>>>> ...
> > >> > >
> > >> > > --
> > >> > > Signature
> > >> > > <http://www.openbet.com/>     Michal Borowiecki
> > >> > > Senior Software Engineer L4
> > >> > >       T:      +44 208 742 1600 <+44%2020%208742%201600>
> > >> > >
> > >> > >
> > >> > >       +44 203 249 8448 <+44%2020%203249%208448>
> > >> > >
> > >> > >
> > >> > >
> > >> > >       E:      michal.borowiecki@openbet.com
> > >> > >       W:      www.openbet.com <http://www.openbet.com/>
> > >> > >
> > >> > >
> > >> > >       OpenBet Ltd
> > >> > >
> > >> > >       Chiswick Park Building 9
> > >> > >
> > >> > >       566 Chiswick High Rd
> > >> > >
> > >> > >       London
> > >> > >
> > >> > >       W4 5XT
> > >> > >
> > >> > >       UK
> > >> > >
> > >> > >
> > >> > > <https://www.openbet.com/email_promo>
> > >> > >
> > >> > > This message is confidential and intended only for the addressee.
> If
> > >> you
> > >> > > have received this message in error, please immediately notify the
> > >> > > postmaster@openbet.com <ma...@openbet.com> and delete
> > it
> > >> > > from your system as well as any copies. The content of e-mails as
> > well
> > >> > > as traffic data may be monitored by OpenBet for employment and
> > >> security
> > >> > > purposes. To protect the environment please do not print this
> e-mail
> > >> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> > >> Building
> > >> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A
> company
> > >> > > registered in England and Wales. Registered no. 3134634. VAT no.
> > >> > > GB927523612
> > >> > >
> > >> >
> > >> >
> > >>
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Xavier Léauté <xa...@confluent.io>.
+1 from me

any stream should be able to initialize the cogroup

On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman <wi...@gmail.com>
wrote:

> I will update the kip to have only the aggregator in the first cogroup call
> and the initializer and serde in the aggregate calls.
>
> This seems to be the consensus on the email chain.
>
> Thanks,
> Kyle
>
> On Jun 14, 2017 5:41 PM, wrote:
>
> That is not the case. No matter which stream the record comes in on the
> initializer is called if there is not currently an object in the store.
>
> On Jun 14, 2017 5:12 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
>
> While regarding where we should ask users to set serdes: I think I'm
> convinced by Xavier that they should be in the `aggregate` call (but only
> those does not pass in a state store supplier) instead of the
> `KStream#cogroup` call to be consistent with other aggregate functions.
>
> BTW another motivation for me to suggest keeping the initializer on the
> first stream is that by reviewing the PR (some time ago though, so again I
> might be wrong) we will trigger the initializer only when we received an
> incoming record from the first stream whose key is not in the state store
> yet, while for other streams we will just drop it on the floor. If that is
> actually not the case, that we call initializer on any one of the
> co-grouped streams' incoming records, then I'm open to set the initializer
> at the `aggregate` call as well.
>
>
> Guozhang
>
> On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > I'd suggest we do not block this KIP until the serde work has been sorted
> > out: we cannot estimate yet how long it will take yet. Instead let's say
> > make an agreement on where we want to specify the serdes: whether on the
> > first co-group call or on the aggregate call.
> >
> > Also about the initializer specification I actually felt that the first
> > cogrouped stream is special (Kyle please feel free to correct me if I'm
> > wrong) and that is why I thought it is better to specify the initializer
> at
> > the beginning: since from the typing you can see that the final
> aggregated
> > value type is defined to be the same as the first co-grouped stream, and
> > for any intermediate stream to co-group, their value types not be
> inherited
> > but the value be "incorporated" into the original stream:
> >
> >  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> > groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
> >
> > Note that we do not have a cogroup function that returns
> > CogroupedKStream<K, T>.
> >
> >
> > Guozhang
> >
> >
> > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:
> >
> >> +1 on deferring discussion on Serdes until API improvements are ironed
> >> out.
> >>
> >> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <matthias@confluent.io
> >
> >> wrote:
> >>
> >> > Hi,
> >> >
> >> > I am just catching up on this thread. (1) as most people agree, we
> >> > should not add anything to KStreamBuilder (btw: we actually plan to
> move
> >> > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
> >> > unnatural API atm).
> >> >
> >> > About specifying Serdes: there is still the idea to improve to overall
> >> > API from the current "we are adding more overloads"-pattern to a
> >> > builder-like pattern. This might make the whole discussion void if we
> do
> >> > this. Thus, it might make sense to keep this in mind (or even delay
> this
> >> > KIP?). It seems a waste of time to discuss all this if we are going to
> >> > chance it in 2 month anyway... Just saying.
> >> >
> >> >
> >> > -Matthias
> >> >
> >> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> >> > > You're right, I haven't thought of that.
> >> > >
> >> > > Cheers,
> >> > >
> >> > > Michał
> >> > >
> >> > >
> >> > > On 13/06/17 13:00, Kyle Winkelman wrote:
> >> > >> First, I would prefer not calling it aggregate because there are
> >> already
> >> > >> plenty of aggregate methods.
> >> > >>
> >> > >> Second, I dont think this would really work because after each
> >> aggregate
> >> > >> you now have a unique KTable (someone may want a table with 4
> streams
> >> > and
> >> > >> reuse those 4 in another table but with one more stream added) and
> >> > unless
> >> > >> we completely duplicate everything every time this isnt really
> >> possible.
> >> > >> Additionally, the cogroup way just requires 1 more call to create
> two
> >> > >> different tables (normal, windowed, and session windowed) this new
> >> way
> >> > >> would require copying the aggregate chain.
> >> > >>
> >> > >> Another way to think about it is with cogroup we know that when
> they
> >> > call
> >> > >> aggregate they arent going to be adding any more aggregators to
> that
> >> > table
> >> > >> but your way requires us to assume they are done adding aggregators
> >> > after
> >> > >> each call so we must return a ktable just to possibly not need to
> >> have
> >> > >> created it.
> >> > >>
> >> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> >> > michal.borowiecki@openbet.com>
> >> > >> wrote:
> >> > >>
> >> > >>> Actually, just had a thought. It started with naming.
> >> > >>>
> >> > >>> Are we actually co-grouping these streams or are we co-aggregating
> >> > them?
> >> > >>>
> >> > >>> After all, in each of the cogroup calls we are providing an
> >> Aggregator
> >> > >>> implementation.
> >> > >>>
> >> > >>>
> >> > >>> If they are really co-aggregated, why don't we turn this around:
> >> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> >> > groupByKey();
> >> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> >> > groupByKey();
> >> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> >> > groupByKey();
> >> > >>>
> >> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1,
> aggregator1,
> >> > >>> aggValueSerde1) // this is the unchanged aggregate method
> >> > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
> >> > >>>         .aggregate(grouped3, aggregator3); // ditto
> >> > >>>
> >> > >>> This means instead of adding cogroup methods on KGroupStream
> >> interface,
> >> > >>> adding aggregate method on KTable interface.
> >> > >>>
> >> > >>> Is that feasible?
> >> > >>>
> >> > >>> Cheers,
> >> > >>> Michał
> >> > >>>
> >> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> >> > >>>
> >> > >>> Also, I still feel that putting initializer on the first cogroup
> can
> >> > >>> mislead users into thinking the first stream is in some way
> special.
> >> > >>> Just my 5c.
> >> > >>> Michał
> >> > >>>
> >> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> >> > >>>
> >> > >>> Agree completely with the argument for serdes belonging in the
> same
> >> > place
> >> > >>> as the state store name, which is in the aggregate method.
> >> > >>>
> >> > >>> Cheers,
> >> > >>>
> >> > >>> Michał
> >> > >>>
> >> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> >> > >>>
> >> > >>> I think we are discussing two separate things here, so it might be
> >> > worth
> >> > >>> clarifying:
> >> > >>>
> >> > >>> 1) the position of the initializer with respect to the
> aggregators.
> >> If
> >> > I
> >> > >>> understand correctly, Guozhang seems to think it is more natural
> to
> >> > specify
> >> > >>> the initializer first, despite it not bearing any relation to the
> >> first
> >> > >>> aggregator. I can see the argument for specifying the initializer
> >> > first,
> >> > >>> but I think it is debatable whether mixing it into the first
> cogroup
> >> > call
> >> > >>> leads to a cleaner API or not.
> >> > >>>
> >> > >>> 2) where the serde should be defined (if necessary). Looking at
> our
> >> > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
> >> > >>> methods. The first one takes the name of the store and associated
> >> > aggregate
> >> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> >> initializer,
> >> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> >> > aggValueSerde,
> >> > >>> String queryableStoreName)
> >> > >>> The second one only takes a state store supplier, and does not
> >> specify
> >> > any
> >> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> >> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator,
> final
> >> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> >> > >>> Presumably, when specifying a state store supplier it shouldn't be
> >> > >>> necessary to specify an aggregate value serde, since the provided
> >> > >>> statestore might not need to serialize the values (e.g. it may
> just
> >> > keep
> >> > >>> them as regular objects in heap) or it may have its own
> >> > >>> internal serialization format.
> >> > >>>
> >> > >>> For consistency I think it would be valuable to preserve the same
> >> two
> >> > >>> aggregate methods for cogroup as well. Since the serde is only
> >> > required in
> >> > >>> one of the two cases, I believe the serde has no place in the
> first
> >> > >>> cogroup() call and should only have to be specified as part of the
> >> > >>> aggregate() method that takes a state store name. In the case of a
> >> > state
> >> > >>> store supplier, no serde would be necessary.
> >> > >>>
> >> > >>>
> >> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wangguoz@gmail.com
> >
> >> > wrote:
> >> > >>>
> >> > >>>> I'd agree that the aggregate value serde and the initializer does
> >> not
> >> > >>>> bear direct relationship with the first `cogroup` calls, but
> after
> >> I
> >> > tried
> >> > >>>> to write some example code with these two different set of APIs I
> >> > felt the
> >> > >>>> current APIs just program more naturally.
> >> > >>>>
> >> > >>>> I know it is kinda subjective, but I do think that user
> experience
> >> > may be
> >> > >>>> more important as a deciding factor than the logical argument for
> >> > public
> >> > >>>> interfaces. So I'd recommend people to also try out writing some
> >> > example
> >> > >>>> lines also and we can circle back and discuss which one feels
> more
> >> > natural
> >> > >>>> to write code.
> >> > >>>>
> >> > >>>>
> >> > >>>> Guozhang
> >> > >>>>
> >> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> >> > >>>> michal.borowiecki@openbet.com> wrote:
> >> > >>>>
> >> > >>>>> I feel it would make more sense to move the initializer and
> serde
> >> to
> >> > the
> >> > >>>>> final aggregate statement, since the serde only applies to the
> >> state
> >> > >>>>> store,
> >> > >>>>> and the initializer doesn't bear any relation to the first group
> >> in
> >> > >>>>> particular.
> >> > >>>>>
> >> > >>>>> +1 for moving initializer and serde from cogroup() to the
> >> aggregate()
> >> > >>>>> for the reasons mentioned above.
> >> > >>>>>
> >> > >>>>> Cheers,
> >> > >>>>>
> >> > >>>>> Michał
> >> > >>>>>
> >> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> >> > >>>>>
> >> > >>>> Note that although the internal `AbstractStoreSupplier` does
> >> maintain
> >> > the
> >> > >>>>> key-value serdes, we do not enforce the interface of
> >> > `StateStoreSupplier`
> >> > >>>>> to always retain that information, and hence we cannot assume
> that
> >> > >>>>> StateStoreSuppliers always retain key / value serdes.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> >> xavier@confluent.io>
> >> > <xa...@confluent.io> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Another reason for the serde not to be in the first cogroup
> call,
> >> is
> >> > that
> >> > >>>>> the serde should not be required if you pass a
> StateStoreSupplier
> >> to
> >> > >>>>> aggregate()
> >> > >>>>>
> >> > >>>>> Regarding the aggregated type <T> I don't the why initializer
> >> should
> >> > be
> >> > >>>>> favored over aggregator to define the type. In my mind
> separating
> >> the
> >> > >>>>> initializer into the last aggregate call clearly indicates that
> >> the
> >> > >>>>> initializer is independent of any of the aggregators or streams
> >> and
> >> > that we
> >> > >>>>> don't wait for grouped1 events to initialize the co-group.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <
> wangguoz@gmail.com
> >
> >> <
> >> > wangguoz@gmail.com> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On a second thought... This is the current proposal API
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> ```
> >> > >>>>>
> >> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> >> initializer,
> >> > >>>>>
> >> > >>>>> final
> >> > >>>>>
> >> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> >> > >>>>> aggValueSerde)
> >> > >>>>>
> >> > >>>>> ```
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> If we do not have the initializer in the first co-group it might
> >> be
> >> > a bit
> >> > >>>>> awkward for users to specify the aggregator that returns a typed
> >> <T>
> >> > >>>>>
> >> > >>>>> value?
> >> > >>>>>
> >> > >>>>> Maybe it is still better to put these two functions in the same
> >> api?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >> > <wa...@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> This suggestion lgtm. I would vote for the first alternative
> than
> >> > >>>>>
> >> > >>>>> adding
> >> > >>>>>
> >> > >>>>> it to the `KStreamBuilder` though.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> >> xavier@confluent.io>
> >> > <xa...@confluent.io>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> I have a minor suggestion to make the API a little bit more
> >> > symmetric.
> >> > >>>>> I feel it would make more sense to move the initializer and
> serde
> >> to
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> final aggregate statement, since the serde only applies to the
> >> state
> >> > >>>>> store,
> >> > >>>>> and the initializer doesn't bear any relation to the first group
> >> in
> >> > >>>>> particular. It would end up looking like this:
> >> > >>>>>
> >> > >>>>> KTable<K, CG> cogrouped =
> >> > >>>>>     grouped1.cogroup(aggregator1)
> >> > >>>>>             .cogroup(grouped2, aggregator2)
> >> > >>>>>             .cogroup(grouped3, aggregator3)
> >> > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
> >> > >>>>>
> >> > >>>>> Alternatively, we could move the the first cogroup() method to
> >> > >>>>> KStreamBuilder, similar to how we have .merge()
> >> > >>>>> and end up with an api that would be even more symmetric.
> >> > >>>>>
> >> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> >> > >>>>>               .cogroup(grouped2, aggregator2)
> >> > >>>>>               .cogroup(grouped3, aggregator3)
> >> > >>>>>               .aggregate(initializer1, aggValueSerde,
> storeName1);
> >> > >>>>>
> >> > >>>>> This doesn't have to be a blocker, but I thought it would make
> the
> >> > API
> >> > >>>>> just
> >> > >>>>> a tad cleaner.
> >> > >>>>>
> >> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <
> wangguoz@gmail.com>
> >> <
> >> > wangguoz@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Kyle,
> >> > >>>>>
> >> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> <
> >> > jim@jagunet.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> This makes much more sense to me. +1
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> I have updated the KIP and my PR. Let me know what you think.
> >> > >>>>> To created a cogrouped stream just call cogroup on a
> >> > >>>>>
> >> > >>>>> KgroupedStream
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> >> > >>>>>
> >> > >>>>> continue
> >> > >>>>>
> >> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> >> > >>>>>
> >> > >>>>> many
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> calls to create a KTable.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> >> > damian.guy@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Hi Kyle,
> >> > >>>>>
> >> > >>>>> Thanks for the update. I think just one initializer makes sense
> >> > >>>>>
> >> > >>>>> as
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> should only be called once per key and generally it is just
> >> > >>>>>
> >> > >>>>> going
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> create
> >> > >>>>>
> >> > >>>>> a new instance of whatever the Aggregate class is.
> >> > >>>>>
> >> > >>>>> Cheers,
> >> > >>>>> Damian
> >> > >>>>>
> >> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Hello all,
> >> > >>>>>
> >> > >>>>> I have spent some more time on this and the best alternative I
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> come
> >> > >>>>>
> >> > >>>>> up
> >> > >>>>>
> >> > >>>>> with is:
> >> > >>>>> KGroupedStream has a single cogroup call that takes an
> >> > >>>>>
> >> > >>>>> initializer
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> an
> >> > >>>>>
> >> > >>>>> aggregator.
> >> > >>>>> CogroupedKStream has a cogroup call that takes additional
> >> > >>>>>
> >> > >>>>> groupedStream
> >> > >>>>>
> >> > >>>>> aggregator pairs.
> >> > >>>>> CogroupedKStream has multiple aggregate methods that create
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> different
> >> > >>>>>
> >> > >>>>> stores.
> >> > >>>>>
> >> > >>>>> I plan on updating the kip but I want people's input on if we
> >> > >>>>>
> >> > >>>>> should
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> the initializer be passed in once at the beginning or if we
> >> > >>>>>
> >> > >>>>> should
> >> > >>>>>
> >> > >>>>> instead
> >> > >>>>>
> >> > >>>>> have the initializer be required for each call to one of the
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> calls. The first makes more sense to me but doesnt allow the
> >> > >>>>>
> >> > >>>>> user
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> specify different initializers for different tables.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Yea I really like that idea I'll see what I can do to update
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> kip
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> my pr when I have some time. I'm not sure how well creating
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> kstreamaggregates will go though because at that point I will
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> thrown
> >> > >>>>>
> >> > >>>>> away the type of the values. It will be type safe I just may
> >> > >>>>>
> >> > >>>>> need to
> >> > >>>>>
> >> > >>>>> do a
> >> > >>>>>
> >> > >>>>> little forcing.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Kyle,
> >> > >>>>>
> >> > >>>>> Thanks for the explanations, my previous read on the wiki
> >> > >>>>>
> >> > >>>>> examples
> >> > >>>>>
> >> > >>>>> was
> >> > >>>>>
> >> > >>>>> wrong.
> >> > >>>>>
> >> > >>>>> So I guess my motivation should be "reduced" to: can we move
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> window
> >> > >>>>>
> >> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> >> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> >> > >>>>>
> >> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> 3
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> 2.
> >> > >>>>>
> >> > >>>>> 2. major: this is for extensibility of the APIs, and since
> >> > >>>>>
> >> > >>>>> we
> >> > >>>>>
> >> > >>>>> are
> >> > >>>>>
> >> > >>>>> removing
> >> > >>>>>
> >> > >>>>> the "Evolving" annotations on Streams it may be harder to
> >> > >>>>>
> >> > >>>>> change it
> >> > >>>>>
> >> > >>>>> again
> >> > >>>>>
> >> > >>>>> in the future. The extended use cases are that people wanted
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> windowed running aggregates on different granularities, e.g.
> >> > >>>>>
> >> > >>>>> "give
> >> > >>>>>
> >> > >>>>> me
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> counts per-minute, per-hour, per-day and per-week", and
> >> > >>>>>
> >> > >>>>> today
> >> > >>>>>
> >> > >>>>> in
> >> > >>>>>
> >> > >>>>> DSL
> >> > >>>>>
> >> > >>>>> we
> >> > >>>>>
> >> > >>>>> need to specify that case in multiple aggregate operators,
> >> > >>>>>
> >> > >>>>> which
> >> > >>>>>
> >> > >>>>> gets
> >> > >>>>>
> >> > >>>>> a
> >> > >>>>>
> >> > >>>>> state store / changelog, etc. And it is possible to optimize
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> as
> >> > >>>>>
> >> > >>>>> well
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> a single state store. Its implementation would be tricky as
> >> > >>>>>
> >> > >>>>> you
> >> > >>>>>
> >> > >>>>> need
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> contain different lengthed windows within your window store
> >> > >>>>>
> >> > >>>>> but
> >> > >>>>>
> >> > >>>>> just
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> the public API point of view, it could be specified as:
> >> > >>>>>
> >> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >> > >>>>> "state-store-name");
> >> > >>>>>
> >> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> >> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> >> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> >> > >>>>>
> >> > >>>>> while underlying we are only using a single store
> >> > >>>>>
> >> > >>>>> "state-store-name"
> >> > >>>>>
> >> > >>>>> for
> >> > >>>>>
> >> > >>>>> it.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Although this feature is out of the scope of this KIP, I'd
> >> > >>>>>
> >> > >>>>> like
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> discuss
> >> > >>>>>
> >> > >>>>> if we can "leave the door open" to make such changes without
> >> > >>>>>
> >> > >>>>> modifying
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> public APIs .
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> I allow defining a single window/sessionwindow one time
> >> > >>>>>
> >> > >>>>> when
> >> > >>>>>
> >> > >>>>> you
> >> > >>>>>
> >> > >>>>> make
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> cogroup call from a KGroupedStream. From then on you are
> >> > >>>>>
> >> > >>>>> using
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> cogroup
> >> > >>>>>
> >> > >>>>> call from with in CogroupedKStream which doesnt accept any
> >> > >>>>>
> >> > >>>>> additional
> >> > >>>>>
> >> > >>>>> windows/sessionwindows.
> >> > >>>>>
> >> > >>>>> Is this what you meant by your question or did I
> >> > >>>>>
> >> > >>>>> misunderstand?
> >> > >>>>>
> >> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> >> > >>>>>
> >> > >>>>> wangguoz@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Another question that came to me is on "window alignment":
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> KIP
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> seems you are allowing users to specify a (potentially
> >> > >>>>>
> >> > >>>>> different)
> >> > >>>>>
> >> > >>>>> window
> >> > >>>>>
> >> > >>>>> spec in each co-grouped input stream. So if these window
> >> > >>>>>
> >> > >>>>> specs
> >> > >>>>>
> >> > >>>>> are
> >> > >>>>>
> >> > >>>>> different how should we "align" them with different input
> >> > >>>>>
> >> > >>>>> streams? I
> >> > >>>>>
> >> > >>>>> think
> >> > >>>>>
> >> > >>>>> it is more natural to only specify on window spec in the
> >> > >>>>>
> >> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> And remove it from the cogroup() functions. WDYT?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> >> > >>>>>
> >> > >>>>> wangguoz@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Thanks for the proposal Kyle, this is a quite common use
> >> > >>>>>
> >> > >>>>> case
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> support
> >> > >>>>>
> >> > >>>>> such multi-way table join (i.e. N source tables with N
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> func)
> >> > >>>>>
> >> > >>>>> with
> >> > >>>>>
> >> > >>>>> a single store and N+1 serdes, I have seen lots of people
> >> > >>>>>
> >> > >>>>> using
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> low-level PAPI to achieve this goal.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> I like your point about not handling other cases such as
> >> > >>>>>
> >> > >>>>> count
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> reduce.
> >> > >>>>>
> >> > >>>>> I think that reduce may not make sense because reduce
> >> > >>>>>
> >> > >>>>> assumes
> >> > >>>>>
> >> > >>>>> that
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> input values are the same as the output values. With
> >> > >>>>>
> >> > >>>>> cogroup
> >> > >>>>>
> >> > >>>>> ...
> >> > >
> >> > > --
> >> > > Signature
> >> > > <http://www.openbet.com/>     Michal Borowiecki
> >> > > Senior Software Engineer L4
> >> > >       T:      +44 208 742 1600 <+44%2020%208742%201600>
> >> > >
> >> > >
> >> > >       +44 203 249 8448 <+44%2020%203249%208448>
> >> > >
> >> > >
> >> > >
> >> > >       E:      michal.borowiecki@openbet.com
> >> > >       W:      www.openbet.com <http://www.openbet.com/>
> >> > >
> >> > >
> >> > >       OpenBet Ltd
> >> > >
> >> > >       Chiswick Park Building 9
> >> > >
> >> > >       566 Chiswick High Rd
> >> > >
> >> > >       London
> >> > >
> >> > >       W4 5XT
> >> > >
> >> > >       UK
> >> > >
> >> > >
> >> > > <https://www.openbet.com/email_promo>
> >> > >
> >> > > This message is confidential and intended only for the addressee. If
> >> you
> >> > > have received this message in error, please immediately notify the
> >> > > postmaster@openbet.com <ma...@openbet.com> and delete
> it
> >> > > from your system as well as any copies. The content of e-mails as
> well
> >> > > as traffic data may be monitored by OpenBet for employment and
> >> security
> >> > > purposes. To protect the environment please do not print this e-mail
> >> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> >> Building
> >> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> >> > > registered in England and Wales. Registered no. 3134634. VAT no.
> >> > > GB927523612
> >> > >
> >> >
> >> >
> >>
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
I will update the kip to have only the aggregator in the first cogroup call
and the initializer and serde in the aggregate calls.

This seems to be the consensus on the email chain.

Thanks,
Kyle

On Jun 14, 2017 5:41 PM, wrote:

That is not the case. No matter which stream the record comes in on the
initializer is called if there is not currently an object in the store.

On Jun 14, 2017 5:12 PM, "Guozhang Wang" <wa...@gmail.com> wrote:

While regarding where we should ask users to set serdes: I think I'm
convinced by Xavier that they should be in the `aggregate` call (but only
those does not pass in a state store supplier) instead of the
`KStream#cogroup` call to be consistent with other aggregate functions.

BTW another motivation for me to suggest keeping the initializer on the
first stream is that by reviewing the PR (some time ago though, so again I
might be wrong) we will trigger the initializer only when we received an
incoming record from the first stream whose key is not in the state store
yet, while for other streams we will just drop it on the floor. If that is
actually not the case, that we call initializer on any one of the
co-grouped streams' incoming records, then I'm open to set the initializer
at the `aggregate` call as well.


Guozhang

On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com> wrote:

> I'd suggest we do not block this KIP until the serde work has been sorted
> out: we cannot estimate yet how long it will take yet. Instead let's say
> make an agreement on where we want to specify the serdes: whether on the
> first co-group call or on the aggregate call.
>
> Also about the initializer specification I actually felt that the first
> cogrouped stream is special (Kyle please feel free to correct me if I'm
> wrong) and that is why I thought it is better to specify the initializer
at
> the beginning: since from the typing you can see that the final aggregated
> value type is defined to be the same as the first co-grouped stream, and
> for any intermediate stream to co-group, their value types not be
inherited
> but the value be "incorporated" into the original stream:
>
>  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream<K, T>.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:
>
>> +1 on deferring discussion on Serdes until API improvements are ironed
>> out.
>>
>> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>> > Hi,
>> >
>> > I am just catching up on this thread. (1) as most people agree, we
>> > should not add anything to KStreamBuilder (btw: we actually plan to
move
>> > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
>> > unnatural API atm).
>> >
>> > About specifying Serdes: there is still the idea to improve to overall
>> > API from the current "we are adding more overloads"-pattern to a
>> > builder-like pattern. This might make the whole discussion void if we
do
>> > this. Thus, it might make sense to keep this in mind (or even delay
this
>> > KIP?). It seems a waste of time to discuss all this if we are going to
>> > chance it in 2 month anyway... Just saying.
>> >
>> >
>> > -Matthias
>> >
>> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
>> > > You're right, I haven't thought of that.
>> > >
>> > > Cheers,
>> > >
>> > > Michał
>> > >
>> > >
>> > > On 13/06/17 13:00, Kyle Winkelman wrote:
>> > >> First, I would prefer not calling it aggregate because there are
>> already
>> > >> plenty of aggregate methods.
>> > >>
>> > >> Second, I dont think this would really work because after each
>> aggregate
>> > >> you now have a unique KTable (someone may want a table with 4
streams
>> > and
>> > >> reuse those 4 in another table but with one more stream added) and
>> > unless
>> > >> we completely duplicate everything every time this isnt really
>> possible.
>> > >> Additionally, the cogroup way just requires 1 more call to create
two
>> > >> different tables (normal, windowed, and session windowed) this new
>> way
>> > >> would require copying the aggregate chain.
>> > >>
>> > >> Another way to think about it is with cogroup we know that when they
>> > call
>> > >> aggregate they arent going to be adding any more aggregators to that
>> > table
>> > >> but your way requires us to assume they are done adding aggregators
>> > after
>> > >> each call so we must return a ktable just to possibly not need to
>> have
>> > >> created it.
>> > >>
>> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
>> > michal.borowiecki@openbet.com>
>> > >> wrote:
>> > >>
>> > >>> Actually, just had a thought. It started with naming.
>> > >>>
>> > >>> Are we actually co-grouping these streams or are we co-aggregating
>> > them?
>> > >>>
>> > >>> After all, in each of the cogroup calls we are providing an
>> Aggregator
>> > >>> implementation.
>> > >>>
>> > >>>
>> > >>> If they are really co-aggregated, why don't we turn this around:
>> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
>> > groupByKey();
>> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
>> > groupByKey();
>> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
>> > groupByKey();
>> > >>>
>> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
>> > >>> aggValueSerde1) // this is the unchanged aggregate method
>> > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
>> > >>>         .aggregate(grouped3, aggregator3); // ditto
>> > >>>
>> > >>> This means instead of adding cogroup methods on KGroupStream
>> interface,
>> > >>> adding aggregate method on KTable interface.
>> > >>>
>> > >>> Is that feasible?
>> > >>>
>> > >>> Cheers,
>> > >>> Michał
>> > >>>
>> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
>> > >>>
>> > >>> Also, I still feel that putting initializer on the first cogroup
can
>> > >>> mislead users into thinking the first stream is in some way
special.
>> > >>> Just my 5c.
>> > >>> Michał
>> > >>>
>> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
>> > >>>
>> > >>> Agree completely with the argument for serdes belonging in the same
>> > place
>> > >>> as the state store name, which is in the aggregate method.
>> > >>>
>> > >>> Cheers,
>> > >>>
>> > >>> Michał
>> > >>>
>> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
>> > >>>
>> > >>> I think we are discussing two separate things here, so it might be
>> > worth
>> > >>> clarifying:
>> > >>>
>> > >>> 1) the position of the initializer with respect to the aggregators.
>> If
>> > I
>> > >>> understand correctly, Guozhang seems to think it is more natural to
>> > specify
>> > >>> the initializer first, despite it not bearing any relation to the
>> first
>> > >>> aggregator. I can see the argument for specifying the initializer
>> > first,
>> > >>> but I think it is debatable whether mixing it into the first
cogroup
>> > call
>> > >>> leads to a cleaner API or not.
>> > >>>
>> > >>> 2) where the serde should be defined (if necessary). Looking at our
>> > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
>> > >>> methods. The first one takes the name of the store and associated
>> > aggregate
>> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
>> initializer,
>> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
>> > aggValueSerde,
>> > >>> String queryableStoreName)
>> > >>> The second one only takes a state store supplier, and does not
>> specify
>> > any
>> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
>> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
>> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
>> > >>> Presumably, when specifying a state store supplier it shouldn't be
>> > >>> necessary to specify an aggregate value serde, since the provided
>> > >>> statestore might not need to serialize the values (e.g. it may just
>> > keep
>> > >>> them as regular objects in heap) or it may have its own
>> > >>> internal serialization format.
>> > >>>
>> > >>> For consistency I think it would be valuable to preserve the same
>> two
>> > >>> aggregate methods for cogroup as well. Since the serde is only
>> > required in
>> > >>> one of the two cases, I believe the serde has no place in the first
>> > >>> cogroup() call and should only have to be specified as part of the
>> > >>> aggregate() method that takes a state store name. In the case of a
>> > state
>> > >>> store supplier, no serde would be necessary.
>> > >>>
>> > >>>
>> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com>
>> > wrote:
>> > >>>
>> > >>>> I'd agree that the aggregate value serde and the initializer does
>> not
>> > >>>> bear direct relationship with the first `cogroup` calls, but after
>> I
>> > tried
>> > >>>> to write some example code with these two different set of APIs I
>> > felt the
>> > >>>> current APIs just program more naturally.
>> > >>>>
>> > >>>> I know it is kinda subjective, but I do think that user experience
>> > may be
>> > >>>> more important as a deciding factor than the logical argument for
>> > public
>> > >>>> interfaces. So I'd recommend people to also try out writing some
>> > example
>> > >>>> lines also and we can circle back and discuss which one feels more
>> > natural
>> > >>>> to write code.
>> > >>>>
>> > >>>>
>> > >>>> Guozhang
>> > >>>>
>> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
>> > >>>> michal.borowiecki@openbet.com> wrote:
>> > >>>>
>> > >>>>> I feel it would make more sense to move the initializer and serde
>> to
>> > the
>> > >>>>> final aggregate statement, since the serde only applies to the
>> state
>> > >>>>> store,
>> > >>>>> and the initializer doesn't bear any relation to the first group
>> in
>> > >>>>> particular.
>> > >>>>>
>> > >>>>> +1 for moving initializer and serde from cogroup() to the
>> aggregate()
>> > >>>>> for the reasons mentioned above.
>> > >>>>>
>> > >>>>> Cheers,
>> > >>>>>
>> > >>>>> Michał
>> > >>>>>
>> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
>> > >>>>>
>> > >>>> Note that although the internal `AbstractStoreSupplier` does
>> maintain
>> > the
>> > >>>>> key-value serdes, we do not enforce the interface of
>> > `StateStoreSupplier`
>> > >>>>> to always retain that information, and hence we cannot assume
that
>> > >>>>> StateStoreSuppliers always retain key / value serdes.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
>> xavier@confluent.io>
>> > <xa...@confluent.io> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Another reason for the serde not to be in the first cogroup call,
>> is
>> > that
>> > >>>>> the serde should not be required if you pass a StateStoreSupplier
>> to
>> > >>>>> aggregate()
>> > >>>>>
>> > >>>>> Regarding the aggregated type <T> I don't the why initializer
>> should
>> > be
>> > >>>>> favored over aggregator to define the type. In my mind separating
>> the
>> > >>>>> initializer into the last aggregate call clearly indicates that
>> the
>> > >>>>> initializer is independent of any of the aggregators or streams
>> and
>> > that we
>> > >>>>> don't wait for grouped1 events to initialize the co-group.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wangguoz@gmail.com
>
>> <
>> > wangguoz@gmail.com> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> On a second thought... This is the current proposal API
>> > >>>>>
>> > >>>>>
>> > >>>>> ```
>> > >>>>>
>> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
>> initializer,
>> > >>>>>
>> > >>>>> final
>> > >>>>>
>> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>> > >>>>> aggValueSerde)
>> > >>>>>
>> > >>>>> ```
>> > >>>>>
>> > >>>>>
>> > >>>>> If we do not have the initializer in the first co-group it might
>> be
>> > a bit
>> > >>>>> awkward for users to specify the aggregator that returns a typed
>> <T>
>> > >>>>>
>> > >>>>> value?
>> > >>>>>
>> > >>>>> Maybe it is still better to put these two functions in the same
>> api?
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
>> wangguoz@gmail.com>
>> > <wa...@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> This suggestion lgtm. I would vote for the first alternative than
>> > >>>>>
>> > >>>>> adding
>> > >>>>>
>> > >>>>> it to the `KStreamBuilder` though.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
>> xavier@confluent.io>
>> > <xa...@confluent.io>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> I have a minor suggestion to make the API a little bit more
>> > symmetric.
>> > >>>>> I feel it would make more sense to move the initializer and serde
>> to
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> final aggregate statement, since the serde only applies to the
>> state
>> > >>>>> store,
>> > >>>>> and the initializer doesn't bear any relation to the first group
>> in
>> > >>>>> particular. It would end up looking like this:
>> > >>>>>
>> > >>>>> KTable<K, CG> cogrouped =
>> > >>>>>     grouped1.cogroup(aggregator1)
>> > >>>>>             .cogroup(grouped2, aggregator2)
>> > >>>>>             .cogroup(grouped3, aggregator3)
>> > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
>> > >>>>>
>> > >>>>> Alternatively, we could move the the first cogroup() method to
>> > >>>>> KStreamBuilder, similar to how we have .merge()
>> > >>>>> and end up with an api that would be even more symmetric.
>> > >>>>>
>> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>> > >>>>>               .cogroup(grouped2, aggregator2)
>> > >>>>>               .cogroup(grouped3, aggregator3)
>> > >>>>>               .aggregate(initializer1, aggValueSerde,
storeName1);
>> > >>>>>
>> > >>>>> This doesn't have to be a blocker, but I thought it would make
the
>> > API
>> > >>>>> just
>> > >>>>> a tad cleaner.
>> > >>>>>
>> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
>> <
>> > wangguoz@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Kyle,
>> > >>>>>
>> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <
>> > jim@jagunet.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> This makes much more sense to me. +1
>> > >>>>>
>> > >>>>>
>> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> I have updated the KIP and my PR. Let me know what you think.
>> > >>>>> To created a cogrouped stream just call cogroup on a
>> > >>>>>
>> > >>>>> KgroupedStream
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
>> > >>>>>
>> > >>>>> continue
>> > >>>>>
>> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
>> > >>>>>
>> > >>>>> many
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> calls to create a KTable.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
>> > damian.guy@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Hi Kyle,
>> > >>>>>
>> > >>>>> Thanks for the update. I think just one initializer makes sense
>> > >>>>>
>> > >>>>> as
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> should only be called once per key and generally it is just
>> > >>>>>
>> > >>>>> going
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> create
>> > >>>>>
>> > >>>>> a new instance of whatever the Aggregate class is.
>> > >>>>>
>> > >>>>> Cheers,
>> > >>>>> Damian
>> > >>>>>
>> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Hello all,
>> > >>>>>
>> > >>>>> I have spent some more time on this and the best alternative I
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> come
>> > >>>>>
>> > >>>>> up
>> > >>>>>
>> > >>>>> with is:
>> > >>>>> KGroupedStream has a single cogroup call that takes an
>> > >>>>>
>> > >>>>> initializer
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> an
>> > >>>>>
>> > >>>>> aggregator.
>> > >>>>> CogroupedKStream has a cogroup call that takes additional
>> > >>>>>
>> > >>>>> groupedStream
>> > >>>>>
>> > >>>>> aggregator pairs.
>> > >>>>> CogroupedKStream has multiple aggregate methods that create
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> different
>> > >>>>>
>> > >>>>> stores.
>> > >>>>>
>> > >>>>> I plan on updating the kip but I want people's input on if we
>> > >>>>>
>> > >>>>> should
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> the initializer be passed in once at the beginning or if we
>> > >>>>>
>> > >>>>> should
>> > >>>>>
>> > >>>>> instead
>> > >>>>>
>> > >>>>> have the initializer be required for each call to one of the
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> calls. The first makes more sense to me but doesnt allow the
>> > >>>>>
>> > >>>>> user
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> specify different initializers for different tables.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Yea I really like that idea I'll see what I can do to update
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> kip
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> my pr when I have some time. I'm not sure how well creating
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> kstreamaggregates will go though because at that point I will
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> thrown
>> > >>>>>
>> > >>>>> away the type of the values. It will be type safe I just may
>> > >>>>>
>> > >>>>> need to
>> > >>>>>
>> > >>>>> do a
>> > >>>>>
>> > >>>>> little forcing.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Kyle,
>> > >>>>>
>> > >>>>> Thanks for the explanations, my previous read on the wiki
>> > >>>>>
>> > >>>>> examples
>> > >>>>>
>> > >>>>> was
>> > >>>>>
>> > >>>>> wrong.
>> > >>>>>
>> > >>>>> So I guess my motivation should be "reduced" to: can we move
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> window
>> > >>>>>
>> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
>> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>> > >>>>>
>> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> 3
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> 2.
>> > >>>>>
>> > >>>>> 2. major: this is for extensibility of the APIs, and since
>> > >>>>>
>> > >>>>> we
>> > >>>>>
>> > >>>>> are
>> > >>>>>
>> > >>>>> removing
>> > >>>>>
>> > >>>>> the "Evolving" annotations on Streams it may be harder to
>> > >>>>>
>> > >>>>> change it
>> > >>>>>
>> > >>>>> again
>> > >>>>>
>> > >>>>> in the future. The extended use cases are that people wanted
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> windowed running aggregates on different granularities, e.g.
>> > >>>>>
>> > >>>>> "give
>> > >>>>>
>> > >>>>> me
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> counts per-minute, per-hour, per-day and per-week", and
>> > >>>>>
>> > >>>>> today
>> > >>>>>
>> > >>>>> in
>> > >>>>>
>> > >>>>> DSL
>> > >>>>>
>> > >>>>> we
>> > >>>>>
>> > >>>>> need to specify that case in multiple aggregate operators,
>> > >>>>>
>> > >>>>> which
>> > >>>>>
>> > >>>>> gets
>> > >>>>>
>> > >>>>> a
>> > >>>>>
>> > >>>>> state store / changelog, etc. And it is possible to optimize
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> as
>> > >>>>>
>> > >>>>> well
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> a single state store. Its implementation would be tricky as
>> > >>>>>
>> > >>>>> you
>> > >>>>>
>> > >>>>> need
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> contain different lengthed windows within your window store
>> > >>>>>
>> > >>>>> but
>> > >>>>>
>> > >>>>> just
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> the public API point of view, it could be specified as:
>> > >>>>>
>> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>> > >>>>> "state-store-name");
>> > >>>>>
>> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
>> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
>> > >>>>> table3 = stream.aggregate(/*per-day window*/)
>> > >>>>>
>> > >>>>> while underlying we are only using a single store
>> > >>>>>
>> > >>>>> "state-store-name"
>> > >>>>>
>> > >>>>> for
>> > >>>>>
>> > >>>>> it.
>> > >>>>>
>> > >>>>>
>> > >>>>> Although this feature is out of the scope of this KIP, I'd
>> > >>>>>
>> > >>>>> like
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> discuss
>> > >>>>>
>> > >>>>> if we can "leave the door open" to make such changes without
>> > >>>>>
>> > >>>>> modifying
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> public APIs .
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> I allow defining a single window/sessionwindow one time
>> > >>>>>
>> > >>>>> when
>> > >>>>>
>> > >>>>> you
>> > >>>>>
>> > >>>>> make
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> cogroup call from a KGroupedStream. From then on you are
>> > >>>>>
>> > >>>>> using
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> cogroup
>> > >>>>>
>> > >>>>> call from with in CogroupedKStream which doesnt accept any
>> > >>>>>
>> > >>>>> additional
>> > >>>>>
>> > >>>>> windows/sessionwindows.
>> > >>>>>
>> > >>>>> Is this what you meant by your question or did I
>> > >>>>>
>> > >>>>> misunderstand?
>> > >>>>>
>> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>> > >>>>>
>> > >>>>> wangguoz@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Another question that came to me is on "window alignment":
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> KIP
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> seems you are allowing users to specify a (potentially
>> > >>>>>
>> > >>>>> different)
>> > >>>>>
>> > >>>>> window
>> > >>>>>
>> > >>>>> spec in each co-grouped input stream. So if these window
>> > >>>>>
>> > >>>>> specs
>> > >>>>>
>> > >>>>> are
>> > >>>>>
>> > >>>>> different how should we "align" them with different input
>> > >>>>>
>> > >>>>> streams? I
>> > >>>>>
>> > >>>>> think
>> > >>>>>
>> > >>>>> it is more natural to only specify on window spec in the
>> > >>>>>
>> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>> > >>>>>
>> > >>>>>
>> > >>>>> And remove it from the cogroup() functions. WDYT?
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>> > >>>>>
>> > >>>>> wangguoz@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Thanks for the proposal Kyle, this is a quite common use
>> > >>>>>
>> > >>>>> case
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> support
>> > >>>>>
>> > >>>>> such multi-way table join (i.e. N source tables with N
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> func)
>> > >>>>>
>> > >>>>> with
>> > >>>>>
>> > >>>>> a single store and N+1 serdes, I have seen lots of people
>> > >>>>>
>> > >>>>> using
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> low-level PAPI to achieve this goal.
>> > >>>>>
>> > >>>>>
>> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> I like your point about not handling other cases such as
>> > >>>>>
>> > >>>>> count
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> reduce.
>> > >>>>>
>> > >>>>> I think that reduce may not make sense because reduce
>> > >>>>>
>> > >>>>> assumes
>> > >>>>>
>> > >>>>> that
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> input values are the same as the output values. With
>> > >>>>>
>> > >>>>> cogroup
>> > >>>>>
>> > >>>>> ...
>> > >
>> > > --
>> > > Signature
>> > > <http://www.openbet.com/>     Michal Borowiecki
>> > > Senior Software Engineer L4
>> > >       T:      +44 208 742 1600
>> > >
>> > >
>> > >       +44 203 249 8448
>> > >
>> > >
>> > >
>> > >       E:      michal.borowiecki@openbet.com
>> > >       W:      www.openbet.com <http://www.openbet.com/>
>> > >
>> > >
>> > >       OpenBet Ltd
>> > >
>> > >       Chiswick Park Building 9
>> > >
>> > >       566 Chiswick High Rd
>> > >
>> > >       London
>> > >
>> > >       W4 5XT
>> > >
>> > >       UK
>> > >
>> > >
>> > > <https://www.openbet.com/email_promo>
>> > >
>> > > This message is confidential and intended only for the addressee. If
>> you
>> > > have received this message in error, please immediately notify the
>> > > postmaster@openbet.com <ma...@openbet.com> and delete it
>> > > from your system as well as any copies. The content of e-mails as
well
>> > > as traffic data may be monitored by OpenBet for employment and
>> security
>> > > purposes. To protect the environment please do not print this e-mail
>> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>> Building
>> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> > > registered in England and Wales. Registered no. 3134634. VAT no.
>> > > GB927523612
>> > >
>> >
>> >
>>
>
>
>
> --
> -- Guozhang
>



--
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
That is not the case. No matter which stream the record comes in on the
initializer is called if there is not currently an object in the store.

On Jun 14, 2017 5:12 PM, "Guozhang Wang" <wa...@gmail.com> wrote:

> While regarding where we should ask users to set serdes: I think I'm
> convinced by Xavier that they should be in the `aggregate` call (but only
> those does not pass in a state store supplier) instead of the
> `KStream#cogroup` call to be consistent with other aggregate functions.
>
> BTW another motivation for me to suggest keeping the initializer on the
> first stream is that by reviewing the PR (some time ago though, so again I
> might be wrong) we will trigger the initializer only when we received an
> incoming record from the first stream whose key is not in the state store
> yet, while for other streams we will just drop it on the floor. If that is
> actually not the case, that we call initializer on any one of the
> co-grouped streams' incoming records, then I'm open to set the initializer
> at the `aggregate` call as well.
>
>
> Guozhang
>
> On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > I'd suggest we do not block this KIP until the serde work has been sorted
> > out: we cannot estimate yet how long it will take yet. Instead let's say
> > make an agreement on where we want to specify the serdes: whether on the
> > first co-group call or on the aggregate call.
> >
> > Also about the initializer specification I actually felt that the first
> > cogrouped stream is special (Kyle please feel free to correct me if I'm
> > wrong) and that is why I thought it is better to specify the initializer
> at
> > the beginning: since from the typing you can see that the final
> aggregated
> > value type is defined to be the same as the first co-grouped stream, and
> > for any intermediate stream to co-group, their value types not be
> inherited
> > but the value be "incorporated" into the original stream:
> >
> >  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> > groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
> >
> > Note that we do not have a cogroup function that returns
> > CogroupedKStream<K, T>.
> >
> >
> > Guozhang
> >
> >
> > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:
> >
> >> +1 on deferring discussion on Serdes until API improvements are ironed
> >> out.
> >>
> >> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <matthias@confluent.io
> >
> >> wrote:
> >>
> >> > Hi,
> >> >
> >> > I am just catching up on this thread. (1) as most people agree, we
> >> > should not add anything to KStreamBuilder (btw: we actually plan to
> move
> >> > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
> >> > unnatural API atm).
> >> >
> >> > About specifying Serdes: there is still the idea to improve to overall
> >> > API from the current "we are adding more overloads"-pattern to a
> >> > builder-like pattern. This might make the whole discussion void if we
> do
> >> > this. Thus, it might make sense to keep this in mind (or even delay
> this
> >> > KIP?). It seems a waste of time to discuss all this if we are going to
> >> > chance it in 2 month anyway... Just saying.
> >> >
> >> >
> >> > -Matthias
> >> >
> >> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> >> > > You're right, I haven't thought of that.
> >> > >
> >> > > Cheers,
> >> > >
> >> > > Michał
> >> > >
> >> > >
> >> > > On 13/06/17 13:00, Kyle Winkelman wrote:
> >> > >> First, I would prefer not calling it aggregate because there are
> >> already
> >> > >> plenty of aggregate methods.
> >> > >>
> >> > >> Second, I dont think this would really work because after each
> >> aggregate
> >> > >> you now have a unique KTable (someone may want a table with 4
> streams
> >> > and
> >> > >> reuse those 4 in another table but with one more stream added) and
> >> > unless
> >> > >> we completely duplicate everything every time this isnt really
> >> possible.
> >> > >> Additionally, the cogroup way just requires 1 more call to create
> two
> >> > >> different tables (normal, windowed, and session windowed) this new
> >> way
> >> > >> would require copying the aggregate chain.
> >> > >>
> >> > >> Another way to think about it is with cogroup we know that when
> they
> >> > call
> >> > >> aggregate they arent going to be adding any more aggregators to
> that
> >> > table
> >> > >> but your way requires us to assume they are done adding aggregators
> >> > after
> >> > >> each call so we must return a ktable just to possibly not need to
> >> have
> >> > >> created it.
> >> > >>
> >> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> >> > michal.borowiecki@openbet.com>
> >> > >> wrote:
> >> > >>
> >> > >>> Actually, just had a thought. It started with naming.
> >> > >>>
> >> > >>> Are we actually co-grouping these streams or are we co-aggregating
> >> > them?
> >> > >>>
> >> > >>> After all, in each of the cogroup calls we are providing an
> >> Aggregator
> >> > >>> implementation.
> >> > >>>
> >> > >>>
> >> > >>> If they are really co-aggregated, why don't we turn this around:
> >> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> >> > groupByKey();
> >> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> >> > groupByKey();
> >> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> >> > groupByKey();
> >> > >>>
> >> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1,
> aggregator1,
> >> > >>> aggValueSerde1) // this is the unchanged aggregate method
> >> > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
> >> > >>>         .aggregate(grouped3, aggregator3); // ditto
> >> > >>>
> >> > >>> This means instead of adding cogroup methods on KGroupStream
> >> interface,
> >> > >>> adding aggregate method on KTable interface.
> >> > >>>
> >> > >>> Is that feasible?
> >> > >>>
> >> > >>> Cheers,
> >> > >>> Michał
> >> > >>>
> >> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> >> > >>>
> >> > >>> Also, I still feel that putting initializer on the first cogroup
> can
> >> > >>> mislead users into thinking the first stream is in some way
> special.
> >> > >>> Just my 5c.
> >> > >>> Michał
> >> > >>>
> >> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> >> > >>>
> >> > >>> Agree completely with the argument for serdes belonging in the
> same
> >> > place
> >> > >>> as the state store name, which is in the aggregate method.
> >> > >>>
> >> > >>> Cheers,
> >> > >>>
> >> > >>> Michał
> >> > >>>
> >> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> >> > >>>
> >> > >>> I think we are discussing two separate things here, so it might be
> >> > worth
> >> > >>> clarifying:
> >> > >>>
> >> > >>> 1) the position of the initializer with respect to the
> aggregators.
> >> If
> >> > I
> >> > >>> understand correctly, Guozhang seems to think it is more natural
> to
> >> > specify
> >> > >>> the initializer first, despite it not bearing any relation to the
> >> first
> >> > >>> aggregator. I can see the argument for specifying the initializer
> >> > first,
> >> > >>> but I think it is debatable whether mixing it into the first
> cogroup
> >> > call
> >> > >>> leads to a cleaner API or not.
> >> > >>>
> >> > >>> 2) where the serde should be defined (if necessary). Looking at
> our
> >> > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
> >> > >>> methods. The first one takes the name of the store and associated
> >> > aggregate
> >> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> >> initializer,
> >> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> >> > aggValueSerde,
> >> > >>> String queryableStoreName)
> >> > >>> The second one only takes a state store supplier, and does not
> >> specify
> >> > any
> >> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> >> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator,
> final
> >> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> >> > >>> Presumably, when specifying a state store supplier it shouldn't be
> >> > >>> necessary to specify an aggregate value serde, since the provided
> >> > >>> statestore might not need to serialize the values (e.g. it may
> just
> >> > keep
> >> > >>> them as regular objects in heap) or it may have its own
> >> > >>> internal serialization format.
> >> > >>>
> >> > >>> For consistency I think it would be valuable to preserve the same
> >> two
> >> > >>> aggregate methods for cogroup as well. Since the serde is only
> >> > required in
> >> > >>> one of the two cases, I believe the serde has no place in the
> first
> >> > >>> cogroup() call and should only have to be specified as part of the
> >> > >>> aggregate() method that takes a state store name. In the case of a
> >> > state
> >> > >>> store supplier, no serde would be necessary.
> >> > >>>
> >> > >>>
> >> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wangguoz@gmail.com
> >
> >> > wrote:
> >> > >>>
> >> > >>>> I'd agree that the aggregate value serde and the initializer does
> >> not
> >> > >>>> bear direct relationship with the first `cogroup` calls, but
> after
> >> I
> >> > tried
> >> > >>>> to write some example code with these two different set of APIs I
> >> > felt the
> >> > >>>> current APIs just program more naturally.
> >> > >>>>
> >> > >>>> I know it is kinda subjective, but I do think that user
> experience
> >> > may be
> >> > >>>> more important as a deciding factor than the logical argument for
> >> > public
> >> > >>>> interfaces. So I'd recommend people to also try out writing some
> >> > example
> >> > >>>> lines also and we can circle back and discuss which one feels
> more
> >> > natural
> >> > >>>> to write code.
> >> > >>>>
> >> > >>>>
> >> > >>>> Guozhang
> >> > >>>>
> >> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> >> > >>>> michal.borowiecki@openbet.com> wrote:
> >> > >>>>
> >> > >>>>> I feel it would make more sense to move the initializer and
> serde
> >> to
> >> > the
> >> > >>>>> final aggregate statement, since the serde only applies to the
> >> state
> >> > >>>>> store,
> >> > >>>>> and the initializer doesn't bear any relation to the first group
> >> in
> >> > >>>>> particular.
> >> > >>>>>
> >> > >>>>> +1 for moving initializer and serde from cogroup() to the
> >> aggregate()
> >> > >>>>> for the reasons mentioned above.
> >> > >>>>>
> >> > >>>>> Cheers,
> >> > >>>>>
> >> > >>>>> Michał
> >> > >>>>>
> >> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> >> > >>>>>
> >> > >>>> Note that although the internal `AbstractStoreSupplier` does
> >> maintain
> >> > the
> >> > >>>>> key-value serdes, we do not enforce the interface of
> >> > `StateStoreSupplier`
> >> > >>>>> to always retain that information, and hence we cannot assume
> that
> >> > >>>>> StateStoreSuppliers always retain key / value serdes.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> >> xavier@confluent.io>
> >> > <xa...@confluent.io> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Another reason for the serde not to be in the first cogroup
> call,
> >> is
> >> > that
> >> > >>>>> the serde should not be required if you pass a
> StateStoreSupplier
> >> to
> >> > >>>>> aggregate()
> >> > >>>>>
> >> > >>>>> Regarding the aggregated type <T> I don't the why initializer
> >> should
> >> > be
> >> > >>>>> favored over aggregator to define the type. In my mind
> separating
> >> the
> >> > >>>>> initializer into the last aggregate call clearly indicates that
> >> the
> >> > >>>>> initializer is independent of any of the aggregators or streams
> >> and
> >> > that we
> >> > >>>>> don't wait for grouped1 events to initialize the co-group.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <
> wangguoz@gmail.com>
> >> <
> >> > wangguoz@gmail.com> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On a second thought... This is the current proposal API
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> ```
> >> > >>>>>
> >> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> >> initializer,
> >> > >>>>>
> >> > >>>>> final
> >> > >>>>>
> >> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> >> > >>>>> aggValueSerde)
> >> > >>>>>
> >> > >>>>> ```
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> If we do not have the initializer in the first co-group it might
> >> be
> >> > a bit
> >> > >>>>> awkward for users to specify the aggregator that returns a typed
> >> <T>
> >> > >>>>>
> >> > >>>>> value?
> >> > >>>>>
> >> > >>>>> Maybe it is still better to put these two functions in the same
> >> api?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
> >> wangguoz@gmail.com>
> >> > <wa...@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> This suggestion lgtm. I would vote for the first alternative
> than
> >> > >>>>>
> >> > >>>>> adding
> >> > >>>>>
> >> > >>>>> it to the `KStreamBuilder` though.
> >> > >>>>>
> >> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> >> xavier@confluent.io>
> >> > <xa...@confluent.io>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> I have a minor suggestion to make the API a little bit more
> >> > symmetric.
> >> > >>>>> I feel it would make more sense to move the initializer and
> serde
> >> to
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> final aggregate statement, since the serde only applies to the
> >> state
> >> > >>>>> store,
> >> > >>>>> and the initializer doesn't bear any relation to the first group
> >> in
> >> > >>>>> particular. It would end up looking like this:
> >> > >>>>>
> >> > >>>>> KTable<K, CG> cogrouped =
> >> > >>>>>     grouped1.cogroup(aggregator1)
> >> > >>>>>             .cogroup(grouped2, aggregator2)
> >> > >>>>>             .cogroup(grouped3, aggregator3)
> >> > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
> >> > >>>>>
> >> > >>>>> Alternatively, we could move the the first cogroup() method to
> >> > >>>>> KStreamBuilder, similar to how we have .merge()
> >> > >>>>> and end up with an api that would be even more symmetric.
> >> > >>>>>
> >> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> >> > >>>>>               .cogroup(grouped2, aggregator2)
> >> > >>>>>               .cogroup(grouped3, aggregator3)
> >> > >>>>>               .aggregate(initializer1, aggValueSerde,
> storeName1);
> >> > >>>>>
> >> > >>>>> This doesn't have to be a blocker, but I thought it would make
> the
> >> > API
> >> > >>>>> just
> >> > >>>>> a tad cleaner.
> >> > >>>>>
> >> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <
> wangguoz@gmail.com>
> >> <
> >> > wangguoz@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Kyle,
> >> > >>>>>
> >> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> <
> >> > jim@jagunet.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> This makes much more sense to me. +1
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> I have updated the KIP and my PR. Let me know what you think.
> >> > >>>>> To created a cogrouped stream just call cogroup on a
> >> > >>>>>
> >> > >>>>> KgroupedStream
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> >> > >>>>>
> >> > >>>>> continue
> >> > >>>>>
> >> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> >> > >>>>>
> >> > >>>>> many
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> calls to create a KTable.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> >> > damian.guy@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Hi Kyle,
> >> > >>>>>
> >> > >>>>> Thanks for the update. I think just one initializer makes sense
> >> > >>>>>
> >> > >>>>> as
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> should only be called once per key and generally it is just
> >> > >>>>>
> >> > >>>>> going
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> create
> >> > >>>>>
> >> > >>>>> a new instance of whatever the Aggregate class is.
> >> > >>>>>
> >> > >>>>> Cheers,
> >> > >>>>> Damian
> >> > >>>>>
> >> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Hello all,
> >> > >>>>>
> >> > >>>>> I have spent some more time on this and the best alternative I
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> come
> >> > >>>>>
> >> > >>>>> up
> >> > >>>>>
> >> > >>>>> with is:
> >> > >>>>> KGroupedStream has a single cogroup call that takes an
> >> > >>>>>
> >> > >>>>> initializer
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> an
> >> > >>>>>
> >> > >>>>> aggregator.
> >> > >>>>> CogroupedKStream has a cogroup call that takes additional
> >> > >>>>>
> >> > >>>>> groupedStream
> >> > >>>>>
> >> > >>>>> aggregator pairs.
> >> > >>>>> CogroupedKStream has multiple aggregate methods that create
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> different
> >> > >>>>>
> >> > >>>>> stores.
> >> > >>>>>
> >> > >>>>> I plan on updating the kip but I want people's input on if we
> >> > >>>>>
> >> > >>>>> should
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> the initializer be passed in once at the beginning or if we
> >> > >>>>>
> >> > >>>>> should
> >> > >>>>>
> >> > >>>>> instead
> >> > >>>>>
> >> > >>>>> have the initializer be required for each call to one of the
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> calls. The first makes more sense to me but doesnt allow the
> >> > >>>>>
> >> > >>>>> user
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> specify different initializers for different tables.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Yea I really like that idea I'll see what I can do to update
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> kip
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> my pr when I have some time. I'm not sure how well creating
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> kstreamaggregates will go though because at that point I will
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> thrown
> >> > >>>>>
> >> > >>>>> away the type of the values. It will be type safe I just may
> >> > >>>>>
> >> > >>>>> need to
> >> > >>>>>
> >> > >>>>> do a
> >> > >>>>>
> >> > >>>>> little forcing.
> >> > >>>>>
> >> > >>>>> Thanks,
> >> > >>>>> Kyle
> >> > >>>>>
> >> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Kyle,
> >> > >>>>>
> >> > >>>>> Thanks for the explanations, my previous read on the wiki
> >> > >>>>>
> >> > >>>>> examples
> >> > >>>>>
> >> > >>>>> was
> >> > >>>>>
> >> > >>>>> wrong.
> >> > >>>>>
> >> > >>>>> So I guess my motivation should be "reduced" to: can we move
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> window
> >> > >>>>>
> >> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> >> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> >> > >>>>>
> >> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> 3
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> 2.
> >> > >>>>>
> >> > >>>>> 2. major: this is for extensibility of the APIs, and since
> >> > >>>>>
> >> > >>>>> we
> >> > >>>>>
> >> > >>>>> are
> >> > >>>>>
> >> > >>>>> removing
> >> > >>>>>
> >> > >>>>> the "Evolving" annotations on Streams it may be harder to
> >> > >>>>>
> >> > >>>>> change it
> >> > >>>>>
> >> > >>>>> again
> >> > >>>>>
> >> > >>>>> in the future. The extended use cases are that people wanted
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> have
> >> > >>>>>
> >> > >>>>> windowed running aggregates on different granularities, e.g.
> >> > >>>>>
> >> > >>>>> "give
> >> > >>>>>
> >> > >>>>> me
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> counts per-minute, per-hour, per-day and per-week", and
> >> > >>>>>
> >> > >>>>> today
> >> > >>>>>
> >> > >>>>> in
> >> > >>>>>
> >> > >>>>> DSL
> >> > >>>>>
> >> > >>>>> we
> >> > >>>>>
> >> > >>>>> need to specify that case in multiple aggregate operators,
> >> > >>>>>
> >> > >>>>> which
> >> > >>>>>
> >> > >>>>> gets
> >> > >>>>>
> >> > >>>>> a
> >> > >>>>>
> >> > >>>>> state store / changelog, etc. And it is possible to optimize
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> as
> >> > >>>>>
> >> > >>>>> well
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> a single state store. Its implementation would be tricky as
> >> > >>>>>
> >> > >>>>> you
> >> > >>>>>
> >> > >>>>> need
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> contain different lengthed windows within your window store
> >> > >>>>>
> >> > >>>>> but
> >> > >>>>>
> >> > >>>>> just
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> the public API point of view, it could be specified as:
> >> > >>>>>
> >> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >> > >>>>> "state-store-name");
> >> > >>>>>
> >> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> >> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> >> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> >> > >>>>>
> >> > >>>>> while underlying we are only using a single store
> >> > >>>>>
> >> > >>>>> "state-store-name"
> >> > >>>>>
> >> > >>>>> for
> >> > >>>>>
> >> > >>>>> it.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Although this feature is out of the scope of this KIP, I'd
> >> > >>>>>
> >> > >>>>> like
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> discuss
> >> > >>>>>
> >> > >>>>> if we can "leave the door open" to make such changes without
> >> > >>>>>
> >> > >>>>> modifying
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> public APIs .
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> I allow defining a single window/sessionwindow one time
> >> > >>>>>
> >> > >>>>> when
> >> > >>>>>
> >> > >>>>> you
> >> > >>>>>
> >> > >>>>> make
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> cogroup call from a KGroupedStream. From then on you are
> >> > >>>>>
> >> > >>>>> using
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> cogroup
> >> > >>>>>
> >> > >>>>> call from with in CogroupedKStream which doesnt accept any
> >> > >>>>>
> >> > >>>>> additional
> >> > >>>>>
> >> > >>>>> windows/sessionwindows.
> >> > >>>>>
> >> > >>>>> Is this what you meant by your question or did I
> >> > >>>>>
> >> > >>>>> misunderstand?
> >> > >>>>>
> >> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> >> > >>>>>
> >> > >>>>> wangguoz@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Another question that came to me is on "window alignment":
> >> > >>>>>
> >> > >>>>> from
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> KIP
> >> > >>>>>
> >> > >>>>> it
> >> > >>>>>
> >> > >>>>> seems you are allowing users to specify a (potentially
> >> > >>>>>
> >> > >>>>> different)
> >> > >>>>>
> >> > >>>>> window
> >> > >>>>>
> >> > >>>>> spec in each co-grouped input stream. So if these window
> >> > >>>>>
> >> > >>>>> specs
> >> > >>>>>
> >> > >>>>> are
> >> > >>>>>
> >> > >>>>> different how should we "align" them with different input
> >> > >>>>>
> >> > >>>>> streams? I
> >> > >>>>>
> >> > >>>>> think
> >> > >>>>>
> >> > >>>>> it is more natural to only specify on window spec in the
> >> > >>>>>
> >> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> And remove it from the cogroup() functions. WDYT?
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> Guozhang
> >> > >>>>>
> >> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> >> > >>>>>
> >> > >>>>> wangguoz@gmail.com>
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> Thanks for the proposal Kyle, this is a quite common use
> >> > >>>>>
> >> > >>>>> case
> >> > >>>>>
> >> > >>>>> to
> >> > >>>>>
> >> > >>>>> support
> >> > >>>>>
> >> > >>>>> such multi-way table join (i.e. N source tables with N
> >> > >>>>>
> >> > >>>>> aggregate
> >> > >>>>>
> >> > >>>>> func)
> >> > >>>>>
> >> > >>>>> with
> >> > >>>>>
> >> > >>>>> a single store and N+1 serdes, I have seen lots of people
> >> > >>>>>
> >> > >>>>> using
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> low-level PAPI to achieve this goal.
> >> > >>>>>
> >> > >>>>>
> >> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >> > >>>>>
> >> > >>>>> winkelman.kyle@gmail.com
> >> > >>>>>
> >> > >>>>> wrote:
> >> > >>>>>
> >> > >>>>> I like your point about not handling other cases such as
> >> > >>>>>
> >> > >>>>> count
> >> > >>>>>
> >> > >>>>> and
> >> > >>>>>
> >> > >>>>> reduce.
> >> > >>>>>
> >> > >>>>> I think that reduce may not make sense because reduce
> >> > >>>>>
> >> > >>>>> assumes
> >> > >>>>>
> >> > >>>>> that
> >> > >>>>>
> >> > >>>>> the
> >> > >>>>>
> >> > >>>>> input values are the same as the output values. With
> >> > >>>>>
> >> > >>>>> cogroup
> >> > >>>>>
> >> > >>>>> ...
> >> > >
> >> > > --
> >> > > Signature
> >> > > <http://www.openbet.com/>     Michal Borowiecki
> >> > > Senior Software Engineer L4
> >> > >       T:      +44 208 742 1600
> >> > >
> >> > >
> >> > >       +44 203 249 8448
> >> > >
> >> > >
> >> > >
> >> > >       E:      michal.borowiecki@openbet.com
> >> > >       W:      www.openbet.com <http://www.openbet.com/>
> >> > >
> >> > >
> >> > >       OpenBet Ltd
> >> > >
> >> > >       Chiswick Park Building 9
> >> > >
> >> > >       566 Chiswick High Rd
> >> > >
> >> > >       London
> >> > >
> >> > >       W4 5XT
> >> > >
> >> > >       UK
> >> > >
> >> > >
> >> > > <https://www.openbet.com/email_promo>
> >> > >
> >> > > This message is confidential and intended only for the addressee. If
> >> you
> >> > > have received this message in error, please immediately notify the
> >> > > postmaster@openbet.com <ma...@openbet.com> and delete
> it
> >> > > from your system as well as any copies. The content of e-mails as
> well
> >> > > as traffic data may be monitored by OpenBet for employment and
> >> security
> >> > > purposes. To protect the environment please do not print this e-mail
> >> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> >> Building
> >> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> >> > > registered in England and Wales. Registered no. 3134634. VAT no.
> >> > > GB927523612
> >> > >
> >> >
> >> >
> >>
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
While regarding where we should ask users to set serdes: I think I'm
convinced by Xavier that they should be in the `aggregate` call (but only
those does not pass in a state store supplier) instead of the
`KStream#cogroup` call to be consistent with other aggregate functions.

BTW another motivation for me to suggest keeping the initializer on the
first stream is that by reviewing the PR (some time ago though, so again I
might be wrong) we will trigger the initializer only when we received an
incoming record from the first stream whose key is not in the state store
yet, while for other streams we will just drop it on the floor. If that is
actually not the case, that we call initializer on any one of the
co-grouped streams' incoming records, then I'm open to set the initializer
at the `aggregate` call as well.


Guozhang

On Wed, Jun 14, 2017 at 2:23 PM, Guozhang Wang <wa...@gmail.com> wrote:

> I'd suggest we do not block this KIP until the serde work has been sorted
> out: we cannot estimate yet how long it will take yet. Instead let's say
> make an agreement on where we want to specify the serdes: whether on the
> first co-group call or on the aggregate call.
>
> Also about the initializer specification I actually felt that the first
> cogrouped stream is special (Kyle please feel free to correct me if I'm
> wrong) and that is why I thought it is better to specify the initializer at
> the beginning: since from the typing you can see that the final aggregated
> value type is defined to be the same as the first co-grouped stream, and
> for any intermediate stream to co-group, their value types not be inherited
> but the value be "incorporated" into the original stream:
>
>  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream<K, T>.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:
>
>> +1 on deferring discussion on Serdes until API improvements are ironed
>> out.
>>
>> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <ma...@confluent.io>
>> wrote:
>>
>> > Hi,
>> >
>> > I am just catching up on this thread. (1) as most people agree, we
>> > should not add anything to KStreamBuilder (btw: we actually plan to move
>> > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
>> > unnatural API atm).
>> >
>> > About specifying Serdes: there is still the idea to improve to overall
>> > API from the current "we are adding more overloads"-pattern to a
>> > builder-like pattern. This might make the whole discussion void if we do
>> > this. Thus, it might make sense to keep this in mind (or even delay this
>> > KIP?). It seems a waste of time to discuss all this if we are going to
>> > chance it in 2 month anyway... Just saying.
>> >
>> >
>> > -Matthias
>> >
>> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
>> > > You're right, I haven't thought of that.
>> > >
>> > > Cheers,
>> > >
>> > > Michał
>> > >
>> > >
>> > > On 13/06/17 13:00, Kyle Winkelman wrote:
>> > >> First, I would prefer not calling it aggregate because there are
>> already
>> > >> plenty of aggregate methods.
>> > >>
>> > >> Second, I dont think this would really work because after each
>> aggregate
>> > >> you now have a unique KTable (someone may want a table with 4 streams
>> > and
>> > >> reuse those 4 in another table but with one more stream added) and
>> > unless
>> > >> we completely duplicate everything every time this isnt really
>> possible.
>> > >> Additionally, the cogroup way just requires 1 more call to create two
>> > >> different tables (normal, windowed, and session windowed) this new
>> way
>> > >> would require copying the aggregate chain.
>> > >>
>> > >> Another way to think about it is with cogroup we know that when they
>> > call
>> > >> aggregate they arent going to be adding any more aggregators to that
>> > table
>> > >> but your way requires us to assume they are done adding aggregators
>> > after
>> > >> each call so we must return a ktable just to possibly not need to
>> have
>> > >> created it.
>> > >>
>> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
>> > michal.borowiecki@openbet.com>
>> > >> wrote:
>> > >>
>> > >>> Actually, just had a thought. It started with naming.
>> > >>>
>> > >>> Are we actually co-grouping these streams or are we co-aggregating
>> > them?
>> > >>>
>> > >>> After all, in each of the cogroup calls we are providing an
>> Aggregator
>> > >>> implementation.
>> > >>>
>> > >>>
>> > >>> If they are really co-aggregated, why don't we turn this around:
>> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
>> > groupByKey();
>> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
>> > groupByKey();
>> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
>> > groupByKey();
>> > >>>
>> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
>> > >>> aggValueSerde1) // this is the unchanged aggregate method
>> > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
>> > >>>         .aggregate(grouped3, aggregator3); // ditto
>> > >>>
>> > >>> This means instead of adding cogroup methods on KGroupStream
>> interface,
>> > >>> adding aggregate method on KTable interface.
>> > >>>
>> > >>> Is that feasible?
>> > >>>
>> > >>> Cheers,
>> > >>> Michał
>> > >>>
>> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
>> > >>>
>> > >>> Also, I still feel that putting initializer on the first cogroup can
>> > >>> mislead users into thinking the first stream is in some way special.
>> > >>> Just my 5c.
>> > >>> Michał
>> > >>>
>> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
>> > >>>
>> > >>> Agree completely with the argument for serdes belonging in the same
>> > place
>> > >>> as the state store name, which is in the aggregate method.
>> > >>>
>> > >>> Cheers,
>> > >>>
>> > >>> Michał
>> > >>>
>> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
>> > >>>
>> > >>> I think we are discussing two separate things here, so it might be
>> > worth
>> > >>> clarifying:
>> > >>>
>> > >>> 1) the position of the initializer with respect to the aggregators.
>> If
>> > I
>> > >>> understand correctly, Guozhang seems to think it is more natural to
>> > specify
>> > >>> the initializer first, despite it not bearing any relation to the
>> first
>> > >>> aggregator. I can see the argument for specifying the initializer
>> > first,
>> > >>> but I think it is debatable whether mixing it into the first cogroup
>> > call
>> > >>> leads to a cleaner API or not.
>> > >>>
>> > >>> 2) where the serde should be defined (if necessary). Looking at our
>> > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
>> > >>> methods. The first one takes the name of the store and associated
>> > aggregate
>> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
>> initializer,
>> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
>> > aggValueSerde,
>> > >>> String queryableStoreName)
>> > >>> The second one only takes a state store supplier, and does not
>> specify
>> > any
>> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
>> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
>> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
>> > >>> Presumably, when specifying a state store supplier it shouldn't be
>> > >>> necessary to specify an aggregate value serde, since the provided
>> > >>> statestore might not need to serialize the values (e.g. it may just
>> > keep
>> > >>> them as regular objects in heap) or it may have its own
>> > >>> internal serialization format.
>> > >>>
>> > >>> For consistency I think it would be valuable to preserve the same
>> two
>> > >>> aggregate methods for cogroup as well. Since the serde is only
>> > required in
>> > >>> one of the two cases, I believe the serde has no place in the first
>> > >>> cogroup() call and should only have to be specified as part of the
>> > >>> aggregate() method that takes a state store name. In the case of a
>> > state
>> > >>> store supplier, no serde would be necessary.
>> > >>>
>> > >>>
>> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com>
>> > wrote:
>> > >>>
>> > >>>> I'd agree that the aggregate value serde and the initializer does
>> not
>> > >>>> bear direct relationship with the first `cogroup` calls, but after
>> I
>> > tried
>> > >>>> to write some example code with these two different set of APIs I
>> > felt the
>> > >>>> current APIs just program more naturally.
>> > >>>>
>> > >>>> I know it is kinda subjective, but I do think that user experience
>> > may be
>> > >>>> more important as a deciding factor than the logical argument for
>> > public
>> > >>>> interfaces. So I'd recommend people to also try out writing some
>> > example
>> > >>>> lines also and we can circle back and discuss which one feels more
>> > natural
>> > >>>> to write code.
>> > >>>>
>> > >>>>
>> > >>>> Guozhang
>> > >>>>
>> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
>> > >>>> michal.borowiecki@openbet.com> wrote:
>> > >>>>
>> > >>>>> I feel it would make more sense to move the initializer and serde
>> to
>> > the
>> > >>>>> final aggregate statement, since the serde only applies to the
>> state
>> > >>>>> store,
>> > >>>>> and the initializer doesn't bear any relation to the first group
>> in
>> > >>>>> particular.
>> > >>>>>
>> > >>>>> +1 for moving initializer and serde from cogroup() to the
>> aggregate()
>> > >>>>> for the reasons mentioned above.
>> > >>>>>
>> > >>>>> Cheers,
>> > >>>>>
>> > >>>>> Michał
>> > >>>>>
>> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
>> > >>>>>
>> > >>>> Note that although the internal `AbstractStoreSupplier` does
>> maintain
>> > the
>> > >>>>> key-value serdes, we do not enforce the interface of
>> > `StateStoreSupplier`
>> > >>>>> to always retain that information, and hence we cannot assume that
>> > >>>>> StateStoreSuppliers always retain key / value serdes.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
>> xavier@confluent.io>
>> > <xa...@confluent.io> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Another reason for the serde not to be in the first cogroup call,
>> is
>> > that
>> > >>>>> the serde should not be required if you pass a StateStoreSupplier
>> to
>> > >>>>> aggregate()
>> > >>>>>
>> > >>>>> Regarding the aggregated type <T> I don't the why initializer
>> should
>> > be
>> > >>>>> favored over aggregator to define the type. In my mind separating
>> the
>> > >>>>> initializer into the last aggregate call clearly indicates that
>> the
>> > >>>>> initializer is independent of any of the aggregators or streams
>> and
>> > that we
>> > >>>>> don't wait for grouped1 events to initialize the co-group.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com>
>> <
>> > wangguoz@gmail.com> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> On a second thought... This is the current proposal API
>> > >>>>>
>> > >>>>>
>> > >>>>> ```
>> > >>>>>
>> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
>> initializer,
>> > >>>>>
>> > >>>>> final
>> > >>>>>
>> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>> > >>>>> aggValueSerde)
>> > >>>>>
>> > >>>>> ```
>> > >>>>>
>> > >>>>>
>> > >>>>> If we do not have the initializer in the first co-group it might
>> be
>> > a bit
>> > >>>>> awkward for users to specify the aggregator that returns a typed
>> <T>
>> > >>>>>
>> > >>>>> value?
>> > >>>>>
>> > >>>>> Maybe it is still better to put these two functions in the same
>> api?
>> > >>>>>
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
>> wangguoz@gmail.com>
>> > <wa...@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> This suggestion lgtm. I would vote for the first alternative than
>> > >>>>>
>> > >>>>> adding
>> > >>>>>
>> > >>>>> it to the `KStreamBuilder` though.
>> > >>>>>
>> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
>> xavier@confluent.io>
>> > <xa...@confluent.io>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> I have a minor suggestion to make the API a little bit more
>> > symmetric.
>> > >>>>> I feel it would make more sense to move the initializer and serde
>> to
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> final aggregate statement, since the serde only applies to the
>> state
>> > >>>>> store,
>> > >>>>> and the initializer doesn't bear any relation to the first group
>> in
>> > >>>>> particular. It would end up looking like this:
>> > >>>>>
>> > >>>>> KTable<K, CG> cogrouped =
>> > >>>>>     grouped1.cogroup(aggregator1)
>> > >>>>>             .cogroup(grouped2, aggregator2)
>> > >>>>>             .cogroup(grouped3, aggregator3)
>> > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
>> > >>>>>
>> > >>>>> Alternatively, we could move the the first cogroup() method to
>> > >>>>> KStreamBuilder, similar to how we have .merge()
>> > >>>>> and end up with an api that would be even more symmetric.
>> > >>>>>
>> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>> > >>>>>               .cogroup(grouped2, aggregator2)
>> > >>>>>               .cogroup(grouped3, aggregator3)
>> > >>>>>               .aggregate(initializer1, aggValueSerde, storeName1);
>> > >>>>>
>> > >>>>> This doesn't have to be a blocker, but I thought it would make the
>> > API
>> > >>>>> just
>> > >>>>> a tad cleaner.
>> > >>>>>
>> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
>> <
>> > wangguoz@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Kyle,
>> > >>>>>
>> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <
>> > jim@jagunet.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> This makes much more sense to me. +1
>> > >>>>>
>> > >>>>>
>> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> I have updated the KIP and my PR. Let me know what you think.
>> > >>>>> To created a cogrouped stream just call cogroup on a
>> > >>>>>
>> > >>>>> KgroupedStream
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
>> > >>>>>
>> > >>>>> continue
>> > >>>>>
>> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
>> > >>>>>
>> > >>>>> many
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> calls to create a KTable.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
>> > damian.guy@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Hi Kyle,
>> > >>>>>
>> > >>>>> Thanks for the update. I think just one initializer makes sense
>> > >>>>>
>> > >>>>> as
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> should only be called once per key and generally it is just
>> > >>>>>
>> > >>>>> going
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> create
>> > >>>>>
>> > >>>>> a new instance of whatever the Aggregate class is.
>> > >>>>>
>> > >>>>> Cheers,
>> > >>>>> Damian
>> > >>>>>
>> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Hello all,
>> > >>>>>
>> > >>>>> I have spent some more time on this and the best alternative I
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> come
>> > >>>>>
>> > >>>>> up
>> > >>>>>
>> > >>>>> with is:
>> > >>>>> KGroupedStream has a single cogroup call that takes an
>> > >>>>>
>> > >>>>> initializer
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> an
>> > >>>>>
>> > >>>>> aggregator.
>> > >>>>> CogroupedKStream has a cogroup call that takes additional
>> > >>>>>
>> > >>>>> groupedStream
>> > >>>>>
>> > >>>>> aggregator pairs.
>> > >>>>> CogroupedKStream has multiple aggregate methods that create
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> different
>> > >>>>>
>> > >>>>> stores.
>> > >>>>>
>> > >>>>> I plan on updating the kip but I want people's input on if we
>> > >>>>>
>> > >>>>> should
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> the initializer be passed in once at the beginning or if we
>> > >>>>>
>> > >>>>> should
>> > >>>>>
>> > >>>>> instead
>> > >>>>>
>> > >>>>> have the initializer be required for each call to one of the
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> calls. The first makes more sense to me but doesnt allow the
>> > >>>>>
>> > >>>>> user
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> specify different initializers for different tables.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> Yea I really like that idea I'll see what I can do to update
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> kip
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> my pr when I have some time. I'm not sure how well creating
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> kstreamaggregates will go though because at that point I will
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> thrown
>> > >>>>>
>> > >>>>> away the type of the values. It will be type safe I just may
>> > >>>>>
>> > >>>>> need to
>> > >>>>>
>> > >>>>> do a
>> > >>>>>
>> > >>>>> little forcing.
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Kyle
>> > >>>>>
>> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Kyle,
>> > >>>>>
>> > >>>>> Thanks for the explanations, my previous read on the wiki
>> > >>>>>
>> > >>>>> examples
>> > >>>>>
>> > >>>>> was
>> > >>>>>
>> > >>>>> wrong.
>> > >>>>>
>> > >>>>> So I guess my motivation should be "reduced" to: can we move
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> window
>> > >>>>>
>> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
>> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>> > >>>>>
>> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> 3
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> 2.
>> > >>>>>
>> > >>>>> 2. major: this is for extensibility of the APIs, and since
>> > >>>>>
>> > >>>>> we
>> > >>>>>
>> > >>>>> are
>> > >>>>>
>> > >>>>> removing
>> > >>>>>
>> > >>>>> the "Evolving" annotations on Streams it may be harder to
>> > >>>>>
>> > >>>>> change it
>> > >>>>>
>> > >>>>> again
>> > >>>>>
>> > >>>>> in the future. The extended use cases are that people wanted
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> have
>> > >>>>>
>> > >>>>> windowed running aggregates on different granularities, e.g.
>> > >>>>>
>> > >>>>> "give
>> > >>>>>
>> > >>>>> me
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> counts per-minute, per-hour, per-day and per-week", and
>> > >>>>>
>> > >>>>> today
>> > >>>>>
>> > >>>>> in
>> > >>>>>
>> > >>>>> DSL
>> > >>>>>
>> > >>>>> we
>> > >>>>>
>> > >>>>> need to specify that case in multiple aggregate operators,
>> > >>>>>
>> > >>>>> which
>> > >>>>>
>> > >>>>> gets
>> > >>>>>
>> > >>>>> a
>> > >>>>>
>> > >>>>> state store / changelog, etc. And it is possible to optimize
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> as
>> > >>>>>
>> > >>>>> well
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> a single state store. Its implementation would be tricky as
>> > >>>>>
>> > >>>>> you
>> > >>>>>
>> > >>>>> need
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> contain different lengthed windows within your window store
>> > >>>>>
>> > >>>>> but
>> > >>>>>
>> > >>>>> just
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> the public API point of view, it could be specified as:
>> > >>>>>
>> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>> > >>>>> "state-store-name");
>> > >>>>>
>> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
>> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
>> > >>>>> table3 = stream.aggregate(/*per-day window*/)
>> > >>>>>
>> > >>>>> while underlying we are only using a single store
>> > >>>>>
>> > >>>>> "state-store-name"
>> > >>>>>
>> > >>>>> for
>> > >>>>>
>> > >>>>> it.
>> > >>>>>
>> > >>>>>
>> > >>>>> Although this feature is out of the scope of this KIP, I'd
>> > >>>>>
>> > >>>>> like
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> discuss
>> > >>>>>
>> > >>>>> if we can "leave the door open" to make such changes without
>> > >>>>>
>> > >>>>> modifying
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> public APIs .
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>>
>> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>>
>> > >>>>> I allow defining a single window/sessionwindow one time
>> > >>>>>
>> > >>>>> when
>> > >>>>>
>> > >>>>> you
>> > >>>>>
>> > >>>>> make
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> cogroup call from a KGroupedStream. From then on you are
>> > >>>>>
>> > >>>>> using
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> cogroup
>> > >>>>>
>> > >>>>> call from with in CogroupedKStream which doesnt accept any
>> > >>>>>
>> > >>>>> additional
>> > >>>>>
>> > >>>>> windows/sessionwindows.
>> > >>>>>
>> > >>>>> Is this what you meant by your question or did I
>> > >>>>>
>> > >>>>> misunderstand?
>> > >>>>>
>> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>> > >>>>>
>> > >>>>> wangguoz@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Another question that came to me is on "window alignment":
>> > >>>>>
>> > >>>>> from
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> KIP
>> > >>>>>
>> > >>>>> it
>> > >>>>>
>> > >>>>> seems you are allowing users to specify a (potentially
>> > >>>>>
>> > >>>>> different)
>> > >>>>>
>> > >>>>> window
>> > >>>>>
>> > >>>>> spec in each co-grouped input stream. So if these window
>> > >>>>>
>> > >>>>> specs
>> > >>>>>
>> > >>>>> are
>> > >>>>>
>> > >>>>> different how should we "align" them with different input
>> > >>>>>
>> > >>>>> streams? I
>> > >>>>>
>> > >>>>> think
>> > >>>>>
>> > >>>>> it is more natural to only specify on window spec in the
>> > >>>>>
>> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>> > >>>>>
>> > >>>>>
>> > >>>>> And remove it from the cogroup() functions. WDYT?
>> > >>>>>
>> > >>>>>
>> > >>>>> Guozhang
>> > >>>>>
>> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>> > >>>>>
>> > >>>>> wangguoz@gmail.com>
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> Thanks for the proposal Kyle, this is a quite common use
>> > >>>>>
>> > >>>>> case
>> > >>>>>
>> > >>>>> to
>> > >>>>>
>> > >>>>> support
>> > >>>>>
>> > >>>>> such multi-way table join (i.e. N source tables with N
>> > >>>>>
>> > >>>>> aggregate
>> > >>>>>
>> > >>>>> func)
>> > >>>>>
>> > >>>>> with
>> > >>>>>
>> > >>>>> a single store and N+1 serdes, I have seen lots of people
>> > >>>>>
>> > >>>>> using
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> low-level PAPI to achieve this goal.
>> > >>>>>
>> > >>>>>
>> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>> > >>>>>
>> > >>>>> winkelman.kyle@gmail.com
>> > >>>>>
>> > >>>>> wrote:
>> > >>>>>
>> > >>>>> I like your point about not handling other cases such as
>> > >>>>>
>> > >>>>> count
>> > >>>>>
>> > >>>>> and
>> > >>>>>
>> > >>>>> reduce.
>> > >>>>>
>> > >>>>> I think that reduce may not make sense because reduce
>> > >>>>>
>> > >>>>> assumes
>> > >>>>>
>> > >>>>> that
>> > >>>>>
>> > >>>>> the
>> > >>>>>
>> > >>>>> input values are the same as the output values. With
>> > >>>>>
>> > >>>>> cogroup
>> > >>>>>
>> > >>>>> ...
>> > >
>> > > --
>> > > Signature
>> > > <http://www.openbet.com/>     Michal Borowiecki
>> > > Senior Software Engineer L4
>> > >       T:      +44 208 742 1600
>> > >
>> > >
>> > >       +44 203 249 8448
>> > >
>> > >
>> > >
>> > >       E:      michal.borowiecki@openbet.com
>> > >       W:      www.openbet.com <http://www.openbet.com/>
>> > >
>> > >
>> > >       OpenBet Ltd
>> > >
>> > >       Chiswick Park Building 9
>> > >
>> > >       566 Chiswick High Rd
>> > >
>> > >       London
>> > >
>> > >       W4 5XT
>> > >
>> > >       UK
>> > >
>> > >
>> > > <https://www.openbet.com/email_promo>
>> > >
>> > > This message is confidential and intended only for the addressee. If
>> you
>> > > have received this message in error, please immediately notify the
>> > > postmaster@openbet.com <ma...@openbet.com> and delete it
>> > > from your system as well as any copies. The content of e-mails as well
>> > > as traffic data may be monitored by OpenBet for employment and
>> security
>> > > purposes. To protect the environment please do not print this e-mail
>> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
>> Building
>> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
>> > > registered in England and Wales. Registered no. 3134634. VAT no.
>> > > GB927523612
>> > >
>> >
>> >
>>
>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
To clarify it isn't required to have the initializer in the first cogroup
because the first aggregator will have the value type. I like how the
initializer makes it abundantly clear that the final type will be that.
Right now I'm split because the case may be made that you want to supply a
different initializer to different aggregate calls, but like I said it
makes it clear to the user what their final value is going to be.

On Jun 14, 2017 4:23 PM, "Guozhang Wang" <wa...@gmail.com> wrote:

> I'd suggest we do not block this KIP until the serde work has been sorted
> out: we cannot estimate yet how long it will take yet. Instead let's say
> make an agreement on where we want to specify the serdes: whether on the
> first co-group call or on the aggregate call.
>
> Also about the initializer specification I actually felt that the first
> cogrouped stream is special (Kyle please feel free to correct me if I'm
> wrong) and that is why I thought it is better to specify the initializer at
> the beginning: since from the typing you can see that the final aggregated
> value type is defined to be the same as the first co-grouped stream, and
> for any intermediate stream to co-group, their value types not be inherited
> but the value be "incorporated" into the original stream:
>
>  <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
> groupedStream, final Aggregator<? super K, ? super T, V> aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream<K, T>.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:
>
> > +1 on deferring discussion on Serdes until API improvements are ironed
> out.
> >
> > On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> > > Hi,
> > >
> > > I am just catching up on this thread. (1) as most people agree, we
> > > should not add anything to KStreamBuilder (btw: we actually plan to
> move
> > > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
> > > unnatural API atm).
> > >
> > > About specifying Serdes: there is still the idea to improve to overall
> > > API from the current "we are adding more overloads"-pattern to a
> > > builder-like pattern. This might make the whole discussion void if we
> do
> > > this. Thus, it might make sense to keep this in mind (or even delay
> this
> > > KIP?). It seems a waste of time to discuss all this if we are going to
> > > chance it in 2 month anyway... Just saying.
> > >
> > >
> > > -Matthias
> > >
> > > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> > > > You're right, I haven't thought of that.
> > > >
> > > > Cheers,
> > > >
> > > > Michał
> > > >
> > > >
> > > > On 13/06/17 13:00, Kyle Winkelman wrote:
> > > >> First, I would prefer not calling it aggregate because there are
> > already
> > > >> plenty of aggregate methods.
> > > >>
> > > >> Second, I dont think this would really work because after each
> > aggregate
> > > >> you now have a unique KTable (someone may want a table with 4
> streams
> > > and
> > > >> reuse those 4 in another table but with one more stream added) and
> > > unless
> > > >> we completely duplicate everything every time this isnt really
> > possible.
> > > >> Additionally, the cogroup way just requires 1 more call to create
> two
> > > >> different tables (normal, windowed, and session windowed) this new
> way
> > > >> would require copying the aggregate chain.
> > > >>
> > > >> Another way to think about it is with cogroup we know that when they
> > > call
> > > >> aggregate they arent going to be adding any more aggregators to that
> > > table
> > > >> but your way requires us to assume they are done adding aggregators
> > > after
> > > >> each call so we must return a ktable just to possibly not need to
> have
> > > >> created it.
> > > >>
> > > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> > > michal.borowiecki@openbet.com>
> > > >> wrote:
> > > >>
> > > >>> Actually, just had a thought. It started with naming.
> > > >>>
> > > >>> Are we actually co-grouping these streams or are we co-aggregating
> > > them?
> > > >>>
> > > >>> After all, in each of the cogroup calls we are providing an
> > Aggregator
> > > >>> implementation.
> > > >>>
> > > >>>
> > > >>> If they are really co-aggregated, why don't we turn this around:
> > > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> > > groupByKey();
> > > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> > > groupByKey();
> > > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> > > groupByKey();
> > > >>>
> > > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1,
> aggregator1,
> > > >>> aggValueSerde1) // this is the unchanged aggregate method
> > > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
> > > >>>         .aggregate(grouped3, aggregator3); // ditto
> > > >>>
> > > >>> This means instead of adding cogroup methods on KGroupStream
> > interface,
> > > >>> adding aggregate method on KTable interface.
> > > >>>
> > > >>> Is that feasible?
> > > >>>
> > > >>> Cheers,
> > > >>> Michał
> > > >>>
> > > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> > > >>>
> > > >>> Also, I still feel that putting initializer on the first cogroup
> can
> > > >>> mislead users into thinking the first stream is in some way
> special.
> > > >>> Just my 5c.
> > > >>> Michał
> > > >>>
> > > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> > > >>>
> > > >>> Agree completely with the argument for serdes belonging in the same
> > > place
> > > >>> as the state store name, which is in the aggregate method.
> > > >>>
> > > >>> Cheers,
> > > >>>
> > > >>> Michał
> > > >>>
> > > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> > > >>>
> > > >>> I think we are discussing two separate things here, so it might be
> > > worth
> > > >>> clarifying:
> > > >>>
> > > >>> 1) the position of the initializer with respect to the aggregators.
> > If
> > > I
> > > >>> understand correctly, Guozhang seems to think it is more natural to
> > > specify
> > > >>> the initializer first, despite it not bearing any relation to the
> > first
> > > >>> aggregator. I can see the argument for specifying the initializer
> > > first,
> > > >>> but I think it is debatable whether mixing it into the first
> cogroup
> > > call
> > > >>> leads to a cleaner API or not.
> > > >>>
> > > >>> 2) where the serde should be defined (if necessary). Looking at our
> > > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
> > > >>> methods. The first one takes the name of the store and associated
> > > aggregate
> > > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> > initializer,
> > > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> > > aggValueSerde,
> > > >>> String queryableStoreName)
> > > >>> The second one only takes a state store supplier, and does not
> > specify
> > > any
> > > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> > > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
> > > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> > > >>> Presumably, when specifying a state store supplier it shouldn't be
> > > >>> necessary to specify an aggregate value serde, since the provided
> > > >>> statestore might not need to serialize the values (e.g. it may just
> > > keep
> > > >>> them as regular objects in heap) or it may have its own
> > > >>> internal serialization format.
> > > >>>
> > > >>> For consistency I think it would be valuable to preserve the same
> two
> > > >>> aggregate methods for cogroup as well. Since the serde is only
> > > required in
> > > >>> one of the two cases, I believe the serde has no place in the first
> > > >>> cogroup() call and should only have to be specified as part of the
> > > >>> aggregate() method that takes a state store name. In the case of a
> > > state
> > > >>> store supplier, no serde would be necessary.
> > > >>>
> > > >>>
> > > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >>>
> > > >>>> I'd agree that the aggregate value serde and the initializer does
> > not
> > > >>>> bear direct relationship with the first `cogroup` calls, but
> after I
> > > tried
> > > >>>> to write some example code with these two different set of APIs I
> > > felt the
> > > >>>> current APIs just program more naturally.
> > > >>>>
> > > >>>> I know it is kinda subjective, but I do think that user experience
> > > may be
> > > >>>> more important as a deciding factor than the logical argument for
> > > public
> > > >>>> interfaces. So I'd recommend people to also try out writing some
> > > example
> > > >>>> lines also and we can circle back and discuss which one feels more
> > > natural
> > > >>>> to write code.
> > > >>>>
> > > >>>>
> > > >>>> Guozhang
> > > >>>>
> > > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> > > >>>> michal.borowiecki@openbet.com> wrote:
> > > >>>>
> > > >>>>> I feel it would make more sense to move the initializer and serde
> > to
> > > the
> > > >>>>> final aggregate statement, since the serde only applies to the
> > state
> > > >>>>> store,
> > > >>>>> and the initializer doesn't bear any relation to the first group
> in
> > > >>>>> particular.
> > > >>>>>
> > > >>>>> +1 for moving initializer and serde from cogroup() to the
> > aggregate()
> > > >>>>> for the reasons mentioned above.
> > > >>>>>
> > > >>>>> Cheers,
> > > >>>>>
> > > >>>>> Michał
> > > >>>>>
> > > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> > > >>>>>
> > > >>>> Note that although the internal `AbstractStoreSupplier` does
> > maintain
> > > the
> > > >>>>> key-value serdes, we do not enforce the interface of
> > > `StateStoreSupplier`
> > > >>>>> to always retain that information, and hence we cannot assume
> that
> > > >>>>> StateStoreSuppliers always retain key / value serdes.
> > > >>>>>
> > > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> > xavier@confluent.io>
> > > <xa...@confluent.io> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> Another reason for the serde not to be in the first cogroup call,
> > is
> > > that
> > > >>>>> the serde should not be required if you pass a StateStoreSupplier
> > to
> > > >>>>> aggregate()
> > > >>>>>
> > > >>>>> Regarding the aggregated type <T> I don't the why initializer
> > should
> > > be
> > > >>>>> favored over aggregator to define the type. In my mind separating
> > the
> > > >>>>> initializer into the last aggregate call clearly indicates that
> the
> > > >>>>> initializer is independent of any of the aggregators or streams
> and
> > > that we
> > > >>>>> don't wait for grouped1 events to initialize the co-group.
> > > >>>>>
> > > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <
> wangguoz@gmail.com>
> > <
> > > wangguoz@gmail.com> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> On a second thought... This is the current proposal API
> > > >>>>>
> > > >>>>>
> > > >>>>> ```
> > > >>>>>
> > > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> > initializer,
> > > >>>>>
> > > >>>>> final
> > > >>>>>
> > > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> > > >>>>> aggValueSerde)
> > > >>>>>
> > > >>>>> ```
> > > >>>>>
> > > >>>>>
> > > >>>>> If we do not have the initializer in the first co-group it might
> be
> > > a bit
> > > >>>>> awkward for users to specify the aggregator that returns a typed
> > <T>
> > > >>>>>
> > > >>>>> value?
> > > >>>>>
> > > >>>>> Maybe it is still better to put these two functions in the same
> > api?
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > <wa...@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> This suggestion lgtm. I would vote for the first alternative than
> > > >>>>>
> > > >>>>> adding
> > > >>>>>
> > > >>>>> it to the `KStreamBuilder` though.
> > > >>>>>
> > > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> > xavier@confluent.io>
> > > <xa...@confluent.io>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> I have a minor suggestion to make the API a little bit more
> > > symmetric.
> > > >>>>> I feel it would make more sense to move the initializer and serde
> > to
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> final aggregate statement, since the serde only applies to the
> > state
> > > >>>>> store,
> > > >>>>> and the initializer doesn't bear any relation to the first group
> in
> > > >>>>> particular. It would end up looking like this:
> > > >>>>>
> > > >>>>> KTable<K, CG> cogrouped =
> > > >>>>>     grouped1.cogroup(aggregator1)
> > > >>>>>             .cogroup(grouped2, aggregator2)
> > > >>>>>             .cogroup(grouped3, aggregator3)
> > > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
> > > >>>>>
> > > >>>>> Alternatively, we could move the the first cogroup() method to
> > > >>>>> KStreamBuilder, similar to how we have .merge()
> > > >>>>> and end up with an api that would be even more symmetric.
> > > >>>>>
> > > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> > > >>>>>               .cogroup(grouped2, aggregator2)
> > > >>>>>               .cogroup(grouped3, aggregator3)
> > > >>>>>               .aggregate(initializer1, aggValueSerde,
> storeName1);
> > > >>>>>
> > > >>>>> This doesn't have to be a blocker, but I thought it would make
> the
> > > API
> > > >>>>> just
> > > >>>>> a tad cleaner.
> > > >>>>>
> > > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wangguoz@gmail.com
> >
> > <
> > > wangguoz@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> Kyle,
> > > >>>>>
> > > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> > > >>>>>
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>>
> > > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> <
> > > jim@jagunet.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> This makes much more sense to me. +1
> > > >>>>>
> > > >>>>>
> > > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > > >>>>>
> > > >>>>> winkelman.kyle@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> I have updated the KIP and my PR. Let me know what you think.
> > > >>>>> To created a cogrouped stream just call cogroup on a
> > > >>>>>
> > > >>>>> KgroupedStream
> > > >>>>>
> > > >>>>> and
> > > >>>>>
> > > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> > > >>>>>
> > > >>>>> continue
> > > >>>>>
> > > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> > > >>>>>
> > > >>>>> many
> > > >>>>>
> > > >>>>> aggregate
> > > >>>>>
> > > >>>>> calls to create a KTable.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Kyle
> > > >>>>>
> > > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> > > damian.guy@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> Hi Kyle,
> > > >>>>>
> > > >>>>> Thanks for the update. I think just one initializer makes sense
> > > >>>>>
> > > >>>>> as
> > > >>>>>
> > > >>>>> it
> > > >>>>>
> > > >>>>> should only be called once per key and generally it is just
> > > >>>>>
> > > >>>>> going
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> create
> > > >>>>>
> > > >>>>> a new instance of whatever the Aggregate class is.
> > > >>>>>
> > > >>>>> Cheers,
> > > >>>>> Damian
> > > >>>>>
> > > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > > >>>>>
> > > >>>>> winkelman.kyle@gmail.com
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> Hello all,
> > > >>>>>
> > > >>>>> I have spent some more time on this and the best alternative I
> > > >>>>>
> > > >>>>> have
> > > >>>>>
> > > >>>>> come
> > > >>>>>
> > > >>>>> up
> > > >>>>>
> > > >>>>> with is:
> > > >>>>> KGroupedStream has a single cogroup call that takes an
> > > >>>>>
> > > >>>>> initializer
> > > >>>>>
> > > >>>>> and
> > > >>>>>
> > > >>>>> an
> > > >>>>>
> > > >>>>> aggregator.
> > > >>>>> CogroupedKStream has a cogroup call that takes additional
> > > >>>>>
> > > >>>>> groupedStream
> > > >>>>>
> > > >>>>> aggregator pairs.
> > > >>>>> CogroupedKStream has multiple aggregate methods that create
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> different
> > > >>>>>
> > > >>>>> stores.
> > > >>>>>
> > > >>>>> I plan on updating the kip but I want people's input on if we
> > > >>>>>
> > > >>>>> should
> > > >>>>>
> > > >>>>> have
> > > >>>>>
> > > >>>>> the initializer be passed in once at the beginning or if we
> > > >>>>>
> > > >>>>> should
> > > >>>>>
> > > >>>>> instead
> > > >>>>>
> > > >>>>> have the initializer be required for each call to one of the
> > > >>>>>
> > > >>>>> aggregate
> > > >>>>>
> > > >>>>> calls. The first makes more sense to me but doesnt allow the
> > > >>>>>
> > > >>>>> user
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> specify different initializers for different tables.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Kyle
> > > >>>>>
> > > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > > >>>>>
> > > >>>>> winkelman.kyle@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> Yea I really like that idea I'll see what I can do to update
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> kip
> > > >>>>>
> > > >>>>> and
> > > >>>>>
> > > >>>>> my pr when I have some time. I'm not sure how well creating
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> kstreamaggregates will go though because at that point I will
> > > >>>>>
> > > >>>>> have
> > > >>>>>
> > > >>>>> thrown
> > > >>>>>
> > > >>>>> away the type of the values. It will be type safe I just may
> > > >>>>>
> > > >>>>> need to
> > > >>>>>
> > > >>>>> do a
> > > >>>>>
> > > >>>>> little forcing.
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Kyle
> > > >>>>>
> > > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> Kyle,
> > > >>>>>
> > > >>>>> Thanks for the explanations, my previous read on the wiki
> > > >>>>>
> > > >>>>> examples
> > > >>>>>
> > > >>>>> was
> > > >>>>>
> > > >>>>> wrong.
> > > >>>>>
> > > >>>>> So I guess my motivation should be "reduced" to: can we move
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> window
> > > >>>>>
> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > > >>>>>
> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > > >>>>>
> > > >>>>> from
> > > >>>>>
> > > >>>>> 3
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> 2.
> > > >>>>>
> > > >>>>> 2. major: this is for extensibility of the APIs, and since
> > > >>>>>
> > > >>>>> we
> > > >>>>>
> > > >>>>> are
> > > >>>>>
> > > >>>>> removing
> > > >>>>>
> > > >>>>> the "Evolving" annotations on Streams it may be harder to
> > > >>>>>
> > > >>>>> change it
> > > >>>>>
> > > >>>>> again
> > > >>>>>
> > > >>>>> in the future. The extended use cases are that people wanted
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> have
> > > >>>>>
> > > >>>>> windowed running aggregates on different granularities, e.g.
> > > >>>>>
> > > >>>>> "give
> > > >>>>>
> > > >>>>> me
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> counts per-minute, per-hour, per-day and per-week", and
> > > >>>>>
> > > >>>>> today
> > > >>>>>
> > > >>>>> in
> > > >>>>>
> > > >>>>> DSL
> > > >>>>>
> > > >>>>> we
> > > >>>>>
> > > >>>>> need to specify that case in multiple aggregate operators,
> > > >>>>>
> > > >>>>> which
> > > >>>>>
> > > >>>>> gets
> > > >>>>>
> > > >>>>> a
> > > >>>>>
> > > >>>>> state store / changelog, etc. And it is possible to optimize
> > > >>>>>
> > > >>>>> it
> > > >>>>>
> > > >>>>> as
> > > >>>>>
> > > >>>>> well
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> a single state store. Its implementation would be tricky as
> > > >>>>>
> > > >>>>> you
> > > >>>>>
> > > >>>>> need
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> contain different lengthed windows within your window store
> > > >>>>>
> > > >>>>> but
> > > >>>>>
> > > >>>>> just
> > > >>>>>
> > > >>>>> from
> > > >>>>>
> > > >>>>> the public API point of view, it could be specified as:
> > > >>>>>
> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > > >>>>> "state-store-name");
> > > >>>>>
> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > > >>>>>
> > > >>>>> while underlying we are only using a single store
> > > >>>>>
> > > >>>>> "state-store-name"
> > > >>>>>
> > > >>>>> for
> > > >>>>>
> > > >>>>> it.
> > > >>>>>
> > > >>>>>
> > > >>>>> Although this feature is out of the scope of this KIP, I'd
> > > >>>>>
> > > >>>>> like
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> discuss
> > > >>>>>
> > > >>>>> if we can "leave the door open" to make such changes without
> > > >>>>>
> > > >>>>> modifying
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> public APIs .
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>>
> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > > >>>>>
> > > >>>>> winkelman.kyle@gmail.com
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>
> > > >>>>> I allow defining a single window/sessionwindow one time
> > > >>>>>
> > > >>>>> when
> > > >>>>>
> > > >>>>> you
> > > >>>>>
> > > >>>>> make
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> cogroup call from a KGroupedStream. From then on you are
> > > >>>>>
> > > >>>>> using
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> cogroup
> > > >>>>>
> > > >>>>> call from with in CogroupedKStream which doesnt accept any
> > > >>>>>
> > > >>>>> additional
> > > >>>>>
> > > >>>>> windows/sessionwindows.
> > > >>>>>
> > > >>>>> Is this what you meant by your question or did I
> > > >>>>>
> > > >>>>> misunderstand?
> > > >>>>>
> > > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> > > >>>>>
> > > >>>>> wangguoz@gmail.com
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> Another question that came to me is on "window alignment":
> > > >>>>>
> > > >>>>> from
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> KIP
> > > >>>>>
> > > >>>>> it
> > > >>>>>
> > > >>>>> seems you are allowing users to specify a (potentially
> > > >>>>>
> > > >>>>> different)
> > > >>>>>
> > > >>>>> window
> > > >>>>>
> > > >>>>> spec in each co-grouped input stream. So if these window
> > > >>>>>
> > > >>>>> specs
> > > >>>>>
> > > >>>>> are
> > > >>>>>
> > > >>>>> different how should we "align" them with different input
> > > >>>>>
> > > >>>>> streams? I
> > > >>>>>
> > > >>>>> think
> > > >>>>>
> > > >>>>> it is more natural to only specify on window spec in the
> > > >>>>>
> > > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > > >>>>>
> > > >>>>>
> > > >>>>> And remove it from the cogroup() functions. WDYT?
> > > >>>>>
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > > >>>>>
> > > >>>>> wangguoz@gmail.com>
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> Thanks for the proposal Kyle, this is a quite common use
> > > >>>>>
> > > >>>>> case
> > > >>>>>
> > > >>>>> to
> > > >>>>>
> > > >>>>> support
> > > >>>>>
> > > >>>>> such multi-way table join (i.e. N source tables with N
> > > >>>>>
> > > >>>>> aggregate
> > > >>>>>
> > > >>>>> func)
> > > >>>>>
> > > >>>>> with
> > > >>>>>
> > > >>>>> a single store and N+1 serdes, I have seen lots of people
> > > >>>>>
> > > >>>>> using
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> low-level PAPI to achieve this goal.
> > > >>>>>
> > > >>>>>
> > > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > > >>>>>
> > > >>>>> winkelman.kyle@gmail.com
> > > >>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>> I like your point about not handling other cases such as
> > > >>>>>
> > > >>>>> count
> > > >>>>>
> > > >>>>> and
> > > >>>>>
> > > >>>>> reduce.
> > > >>>>>
> > > >>>>> I think that reduce may not make sense because reduce
> > > >>>>>
> > > >>>>> assumes
> > > >>>>>
> > > >>>>> that
> > > >>>>>
> > > >>>>> the
> > > >>>>>
> > > >>>>> input values are the same as the output values. With
> > > >>>>>
> > > >>>>> cogroup
> > > >>>>>
> > > >>>>> ...
> > > >
> > > > --
> > > > Signature
> > > > <http://www.openbet.com/>     Michal Borowiecki
> > > > Senior Software Engineer L4
> > > >       T:      +44 208 742 1600
> > > >
> > > >
> > > >       +44 203 249 8448
> > > >
> > > >
> > > >
> > > >       E:      michal.borowiecki@openbet.com
> > > >       W:      www.openbet.com <http://www.openbet.com/>
> > > >
> > > >
> > > >       OpenBet Ltd
> > > >
> > > >       Chiswick Park Building 9
> > > >
> > > >       566 Chiswick High Rd
> > > >
> > > >       London
> > > >
> > > >       W4 5XT
> > > >
> > > >       UK
> > > >
> > > >
> > > > <https://www.openbet.com/email_promo>
> > > >
> > > > This message is confidential and intended only for the addressee. If
> > you
> > > > have received this message in error, please immediately notify the
> > > > postmaster@openbet.com <ma...@openbet.com> and delete it
> > > > from your system as well as any copies. The content of e-mails as
> well
> > > > as traffic data may be monitored by OpenBet for employment and
> security
> > > > purposes. To protect the environment please do not print this e-mail
> > > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> > Building
> > > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> > > > registered in England and Wales. Registered no. 3134634. VAT no.
> > > > GB927523612
> > > >
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
I'd suggest we do not block this KIP until the serde work has been sorted
out: we cannot estimate yet how long it will take yet. Instead let's say
make an agreement on where we want to specify the serdes: whether on the
first co-group call or on the aggregate call.

Also about the initializer specification I actually felt that the first
cogrouped stream is special (Kyle please feel free to correct me if I'm
wrong) and that is why I thought it is better to specify the initializer at
the beginning: since from the typing you can see that the final aggregated
value type is defined to be the same as the first co-grouped stream, and
for any intermediate stream to co-group, their value types not be inherited
but the value be "incorporated" into the original stream:

 <T> CogroupedKStream<K, V> cogroup(final KGroupedStream<K, T>
groupedStream, final Aggregator<? super K, ? super T, V> aggregator)

Note that we do not have a cogroup function that returns
CogroupedKStream<K, T>.


Guozhang


On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck <bb...@gmail.com> wrote:

> +1 on deferring discussion on Serdes until API improvements are ironed out.
>
> On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <ma...@confluent.io>
> wrote:
>
> > Hi,
> >
> > I am just catching up on this thread. (1) as most people agree, we
> > should not add anything to KStreamBuilder (btw: we actually plan to move
> > #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
> > unnatural API atm).
> >
> > About specifying Serdes: there is still the idea to improve to overall
> > API from the current "we are adding more overloads"-pattern to a
> > builder-like pattern. This might make the whole discussion void if we do
> > this. Thus, it might make sense to keep this in mind (or even delay this
> > KIP?). It seems a waste of time to discuss all this if we are going to
> > chance it in 2 month anyway... Just saying.
> >
> >
> > -Matthias
> >
> > On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> > > You're right, I haven't thought of that.
> > >
> > > Cheers,
> > >
> > > Michał
> > >
> > >
> > > On 13/06/17 13:00, Kyle Winkelman wrote:
> > >> First, I would prefer not calling it aggregate because there are
> already
> > >> plenty of aggregate methods.
> > >>
> > >> Second, I dont think this would really work because after each
> aggregate
> > >> you now have a unique KTable (someone may want a table with 4 streams
> > and
> > >> reuse those 4 in another table but with one more stream added) and
> > unless
> > >> we completely duplicate everything every time this isnt really
> possible.
> > >> Additionally, the cogroup way just requires 1 more call to create two
> > >> different tables (normal, windowed, and session windowed) this new way
> > >> would require copying the aggregate chain.
> > >>
> > >> Another way to think about it is with cogroup we know that when they
> > call
> > >> aggregate they arent going to be adding any more aggregators to that
> > table
> > >> but your way requires us to assume they are done adding aggregators
> > after
> > >> each call so we must return a ktable just to possibly not need to have
> > >> created it.
> > >>
> > >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> > michal.borowiecki@openbet.com>
> > >> wrote:
> > >>
> > >>> Actually, just had a thought. It started with naming.
> > >>>
> > >>> Are we actually co-grouping these streams or are we co-aggregating
> > them?
> > >>>
> > >>> After all, in each of the cogroup calls we are providing an
> Aggregator
> > >>> implementation.
> > >>>
> > >>>
> > >>> If they are really co-aggregated, why don't we turn this around:
> > >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> > groupByKey();
> > >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> > groupByKey();
> > >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> > groupByKey();
> > >>>
> > >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
> > >>> aggValueSerde1) // this is the unchanged aggregate method
> > >>>         .aggregate(grouped2, aggregator2)  // this is a new method
> > >>>         .aggregate(grouped3, aggregator3); // ditto
> > >>>
> > >>> This means instead of adding cogroup methods on KGroupStream
> interface,
> > >>> adding aggregate method on KTable interface.
> > >>>
> > >>> Is that feasible?
> > >>>
> > >>> Cheers,
> > >>> Michał
> > >>>
> > >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> > >>>
> > >>> Also, I still feel that putting initializer on the first cogroup can
> > >>> mislead users into thinking the first stream is in some way special.
> > >>> Just my 5c.
> > >>> Michał
> > >>>
> > >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> > >>>
> > >>> Agree completely with the argument for serdes belonging in the same
> > place
> > >>> as the state store name, which is in the aggregate method.
> > >>>
> > >>> Cheers,
> > >>>
> > >>> Michał
> > >>>
> > >>> On 12/06/17 18:20, Xavier Léauté wrote:
> > >>>
> > >>> I think we are discussing two separate things here, so it might be
> > worth
> > >>> clarifying:
> > >>>
> > >>> 1) the position of the initializer with respect to the aggregators.
> If
> > I
> > >>> understand correctly, Guozhang seems to think it is more natural to
> > specify
> > >>> the initializer first, despite it not bearing any relation to the
> first
> > >>> aggregator. I can see the argument for specifying the initializer
> > first,
> > >>> but I think it is debatable whether mixing it into the first cogroup
> > call
> > >>> leads to a cleaner API or not.
> > >>>
> > >>> 2) where the serde should be defined (if necessary). Looking at our
> > >>> existing APIs in KGroupedStreams, we always offer two aggregate()
> > >>> methods. The first one takes the name of the store and associated
> > aggregate
> > >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR>
> initializer,
> > >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> > aggValueSerde,
> > >>> String queryableStoreName)
> > >>> The second one only takes a state store supplier, and does not
> specify
> > any
> > >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> > >>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
> > >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> > >>> Presumably, when specifying a state store supplier it shouldn't be
> > >>> necessary to specify an aggregate value serde, since the provided
> > >>> statestore might not need to serialize the values (e.g. it may just
> > keep
> > >>> them as regular objects in heap) or it may have its own
> > >>> internal serialization format.
> > >>>
> > >>> For consistency I think it would be valuable to preserve the same two
> > >>> aggregate methods for cogroup as well. Since the serde is only
> > required in
> > >>> one of the two cases, I believe the serde has no place in the first
> > >>> cogroup() call and should only have to be specified as part of the
> > >>> aggregate() method that takes a state store name. In the case of a
> > state
> > >>> store supplier, no serde would be necessary.
> > >>>
> > >>>
> > >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>>
> > >>>> I'd agree that the aggregate value serde and the initializer does
> not
> > >>>> bear direct relationship with the first `cogroup` calls, but after I
> > tried
> > >>>> to write some example code with these two different set of APIs I
> > felt the
> > >>>> current APIs just program more naturally.
> > >>>>
> > >>>> I know it is kinda subjective, but I do think that user experience
> > may be
> > >>>> more important as a deciding factor than the logical argument for
> > public
> > >>>> interfaces. So I'd recommend people to also try out writing some
> > example
> > >>>> lines also and we can circle back and discuss which one feels more
> > natural
> > >>>> to write code.
> > >>>>
> > >>>>
> > >>>> Guozhang
> > >>>>
> > >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> > >>>> michal.borowiecki@openbet.com> wrote:
> > >>>>
> > >>>>> I feel it would make more sense to move the initializer and serde
> to
> > the
> > >>>>> final aggregate statement, since the serde only applies to the
> state
> > >>>>> store,
> > >>>>> and the initializer doesn't bear any relation to the first group in
> > >>>>> particular.
> > >>>>>
> > >>>>> +1 for moving initializer and serde from cogroup() to the
> aggregate()
> > >>>>> for the reasons mentioned above.
> > >>>>>
> > >>>>> Cheers,
> > >>>>>
> > >>>>> Michał
> > >>>>>
> > >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> > >>>>>
> > >>>> Note that although the internal `AbstractStoreSupplier` does
> maintain
> > the
> > >>>>> key-value serdes, we do not enforce the interface of
> > `StateStoreSupplier`
> > >>>>> to always retain that information, and hence we cannot assume that
> > >>>>> StateStoreSuppliers always retain key / value serdes.
> > >>>>>
> > >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <
> xavier@confluent.io>
> > <xa...@confluent.io> wrote:
> > >>>>>
> > >>>>>
> > >>>>> Another reason for the serde not to be in the first cogroup call,
> is
> > that
> > >>>>> the serde should not be required if you pass a StateStoreSupplier
> to
> > >>>>> aggregate()
> > >>>>>
> > >>>>> Regarding the aggregated type <T> I don't the why initializer
> should
> > be
> > >>>>> favored over aggregator to define the type. In my mind separating
> the
> > >>>>> initializer into the last aggregate call clearly indicates that the
> > >>>>> initializer is independent of any of the aggregators or streams and
> > that we
> > >>>>> don't wait for grouped1 events to initialize the co-group.
> > >>>>>
> > >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com>
> <
> > wangguoz@gmail.com> wrote:
> > >>>>>
> > >>>>>
> > >>>>> On a second thought... This is the current proposal API
> > >>>>>
> > >>>>>
> > >>>>> ```
> > >>>>>
> > >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T>
> initializer,
> > >>>>>
> > >>>>> final
> > >>>>>
> > >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> > >>>>> aggValueSerde)
> > >>>>>
> > >>>>> ```
> > >>>>>
> > >>>>>
> > >>>>> If we do not have the initializer in the first co-group it might be
> > a bit
> > >>>>> awkward for users to specify the aggregator that returns a typed
> <T>
> > >>>>>
> > >>>>> value?
> > >>>>>
> > >>>>> Maybe it is still better to put these two functions in the same
> api?
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wangguoz@gmail.com
> >
> > <wa...@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> This suggestion lgtm. I would vote for the first alternative than
> > >>>>>
> > >>>>> adding
> > >>>>>
> > >>>>> it to the `KStreamBuilder` though.
> > >>>>>
> > >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <
> xavier@confluent.io>
> > <xa...@confluent.io>
> > >>>>> wrote:
> > >>>>>
> > >>>>>
> > >>>>> I have a minor suggestion to make the API a little bit more
> > symmetric.
> > >>>>> I feel it would make more sense to move the initializer and serde
> to
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> final aggregate statement, since the serde only applies to the
> state
> > >>>>> store,
> > >>>>> and the initializer doesn't bear any relation to the first group in
> > >>>>> particular. It would end up looking like this:
> > >>>>>
> > >>>>> KTable<K, CG> cogrouped =
> > >>>>>     grouped1.cogroup(aggregator1)
> > >>>>>             .cogroup(grouped2, aggregator2)
> > >>>>>             .cogroup(grouped3, aggregator3)
> > >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
> > >>>>>
> > >>>>> Alternatively, we could move the the first cogroup() method to
> > >>>>> KStreamBuilder, similar to how we have .merge()
> > >>>>> and end up with an api that would be even more symmetric.
> > >>>>>
> > >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> > >>>>>               .cogroup(grouped2, aggregator2)
> > >>>>>               .cogroup(grouped3, aggregator3)
> > >>>>>               .aggregate(initializer1, aggValueSerde, storeName1);
> > >>>>>
> > >>>>> This doesn't have to be a blocker, but I thought it would make the
> > API
> > >>>>> just
> > >>>>> a tad cleaner.
> > >>>>>
> > >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
> <
> > wangguoz@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Kyle,
> > >>>>>
> > >>>>> Thanks a lot for the updated KIP. It looks good to me.
> > >>>>>
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <
> > jim@jagunet.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> This makes much more sense to me. +1
> > >>>>>
> > >>>>>
> > >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >>>>>
> > >>>>> winkelman.kyle@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> I have updated the KIP and my PR. Let me know what you think.
> > >>>>> To created a cogrouped stream just call cogroup on a
> > >>>>>
> > >>>>> KgroupedStream
> > >>>>>
> > >>>>> and
> > >>>>>
> > >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> > >>>>>
> > >>>>> continue
> > >>>>>
> > >>>>> adding kgroupedstreams and aggregators. Then call one of the
> > >>>>>
> > >>>>> many
> > >>>>>
> > >>>>> aggregate
> > >>>>>
> > >>>>> calls to create a KTable.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Kyle
> > >>>>>
> > >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> > damian.guy@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Hi Kyle,
> > >>>>>
> > >>>>> Thanks for the update. I think just one initializer makes sense
> > >>>>>
> > >>>>> as
> > >>>>>
> > >>>>> it
> > >>>>>
> > >>>>> should only be called once per key and generally it is just
> > >>>>>
> > >>>>> going
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> create
> > >>>>>
> > >>>>> a new instance of whatever the Aggregate class is.
> > >>>>>
> > >>>>> Cheers,
> > >>>>> Damian
> > >>>>>
> > >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > >>>>>
> > >>>>> winkelman.kyle@gmail.com
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>>
> > >>>>> Hello all,
> > >>>>>
> > >>>>> I have spent some more time on this and the best alternative I
> > >>>>>
> > >>>>> have
> > >>>>>
> > >>>>> come
> > >>>>>
> > >>>>> up
> > >>>>>
> > >>>>> with is:
> > >>>>> KGroupedStream has a single cogroup call that takes an
> > >>>>>
> > >>>>> initializer
> > >>>>>
> > >>>>> and
> > >>>>>
> > >>>>> an
> > >>>>>
> > >>>>> aggregator.
> > >>>>> CogroupedKStream has a cogroup call that takes additional
> > >>>>>
> > >>>>> groupedStream
> > >>>>>
> > >>>>> aggregator pairs.
> > >>>>> CogroupedKStream has multiple aggregate methods that create
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> different
> > >>>>>
> > >>>>> stores.
> > >>>>>
> > >>>>> I plan on updating the kip but I want people's input on if we
> > >>>>>
> > >>>>> should
> > >>>>>
> > >>>>> have
> > >>>>>
> > >>>>> the initializer be passed in once at the beginning or if we
> > >>>>>
> > >>>>> should
> > >>>>>
> > >>>>> instead
> > >>>>>
> > >>>>> have the initializer be required for each call to one of the
> > >>>>>
> > >>>>> aggregate
> > >>>>>
> > >>>>> calls. The first makes more sense to me but doesnt allow the
> > >>>>>
> > >>>>> user
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> specify different initializers for different tables.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Kyle
> > >>>>>
> > >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > >>>>>
> > >>>>> winkelman.kyle@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>>
> > >>>>> Yea I really like that idea I'll see what I can do to update
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> kip
> > >>>>>
> > >>>>> and
> > >>>>>
> > >>>>> my pr when I have some time. I'm not sure how well creating
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> kstreamaggregates will go though because at that point I will
> > >>>>>
> > >>>>> have
> > >>>>>
> > >>>>> thrown
> > >>>>>
> > >>>>> away the type of the values. It will be type safe I just may
> > >>>>>
> > >>>>> need to
> > >>>>>
> > >>>>> do a
> > >>>>>
> > >>>>> little forcing.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Kyle
> > >>>>>
> > >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Kyle,
> > >>>>>
> > >>>>> Thanks for the explanations, my previous read on the wiki
> > >>>>>
> > >>>>> examples
> > >>>>>
> > >>>>> was
> > >>>>>
> > >>>>> wrong.
> > >>>>>
> > >>>>> So I guess my motivation should be "reduced" to: can we move
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> window
> > >>>>>
> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >>>>>
> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > >>>>>
> > >>>>> from
> > >>>>>
> > >>>>> 3
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> 2.
> > >>>>>
> > >>>>> 2. major: this is for extensibility of the APIs, and since
> > >>>>>
> > >>>>> we
> > >>>>>
> > >>>>> are
> > >>>>>
> > >>>>> removing
> > >>>>>
> > >>>>> the "Evolving" annotations on Streams it may be harder to
> > >>>>>
> > >>>>> change it
> > >>>>>
> > >>>>> again
> > >>>>>
> > >>>>> in the future. The extended use cases are that people wanted
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> have
> > >>>>>
> > >>>>> windowed running aggregates on different granularities, e.g.
> > >>>>>
> > >>>>> "give
> > >>>>>
> > >>>>> me
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> counts per-minute, per-hour, per-day and per-week", and
> > >>>>>
> > >>>>> today
> > >>>>>
> > >>>>> in
> > >>>>>
> > >>>>> DSL
> > >>>>>
> > >>>>> we
> > >>>>>
> > >>>>> need to specify that case in multiple aggregate operators,
> > >>>>>
> > >>>>> which
> > >>>>>
> > >>>>> gets
> > >>>>>
> > >>>>> a
> > >>>>>
> > >>>>> state store / changelog, etc. And it is possible to optimize
> > >>>>>
> > >>>>> it
> > >>>>>
> > >>>>> as
> > >>>>>
> > >>>>> well
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> a single state store. Its implementation would be tricky as
> > >>>>>
> > >>>>> you
> > >>>>>
> > >>>>> need
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> contain different lengthed windows within your window store
> > >>>>>
> > >>>>> but
> > >>>>>
> > >>>>> just
> > >>>>>
> > >>>>> from
> > >>>>>
> > >>>>> the public API point of view, it could be specified as:
> > >>>>>
> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >>>>> "state-store-name");
> > >>>>>
> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > >>>>>
> > >>>>> while underlying we are only using a single store
> > >>>>>
> > >>>>> "state-store-name"
> > >>>>>
> > >>>>> for
> > >>>>>
> > >>>>> it.
> > >>>>>
> > >>>>>
> > >>>>> Although this feature is out of the scope of this KIP, I'd
> > >>>>>
> > >>>>> like
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> discuss
> > >>>>>
> > >>>>> if we can "leave the door open" to make such changes without
> > >>>>>
> > >>>>> modifying
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> public APIs .
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > >>>>>
> > >>>>> winkelman.kyle@gmail.com
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>>
> > >>>>> I allow defining a single window/sessionwindow one time
> > >>>>>
> > >>>>> when
> > >>>>>
> > >>>>> you
> > >>>>>
> > >>>>> make
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> cogroup call from a KGroupedStream. From then on you are
> > >>>>>
> > >>>>> using
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> cogroup
> > >>>>>
> > >>>>> call from with in CogroupedKStream which doesnt accept any
> > >>>>>
> > >>>>> additional
> > >>>>>
> > >>>>> windows/sessionwindows.
> > >>>>>
> > >>>>> Is this what you meant by your question or did I
> > >>>>>
> > >>>>> misunderstand?
> > >>>>>
> > >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> > >>>>>
> > >>>>> wangguoz@gmail.com
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Another question that came to me is on "window alignment":
> > >>>>>
> > >>>>> from
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> KIP
> > >>>>>
> > >>>>> it
> > >>>>>
> > >>>>> seems you are allowing users to specify a (potentially
> > >>>>>
> > >>>>> different)
> > >>>>>
> > >>>>> window
> > >>>>>
> > >>>>> spec in each co-grouped input stream. So if these window
> > >>>>>
> > >>>>> specs
> > >>>>>
> > >>>>> are
> > >>>>>
> > >>>>> different how should we "align" them with different input
> > >>>>>
> > >>>>> streams? I
> > >>>>>
> > >>>>> think
> > >>>>>
> > >>>>> it is more natural to only specify on window spec in the
> > >>>>>
> > >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >>>>>
> > >>>>>
> > >>>>> And remove it from the cogroup() functions. WDYT?
> > >>>>>
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > >>>>>
> > >>>>> wangguoz@gmail.com>
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> Thanks for the proposal Kyle, this is a quite common use
> > >>>>>
> > >>>>> case
> > >>>>>
> > >>>>> to
> > >>>>>
> > >>>>> support
> > >>>>>
> > >>>>> such multi-way table join (i.e. N source tables with N
> > >>>>>
> > >>>>> aggregate
> > >>>>>
> > >>>>> func)
> > >>>>>
> > >>>>> with
> > >>>>>
> > >>>>> a single store and N+1 serdes, I have seen lots of people
> > >>>>>
> > >>>>> using
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> low-level PAPI to achieve this goal.
> > >>>>>
> > >>>>>
> > >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >>>>>
> > >>>>> winkelman.kyle@gmail.com
> > >>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>> I like your point about not handling other cases such as
> > >>>>>
> > >>>>> count
> > >>>>>
> > >>>>> and
> > >>>>>
> > >>>>> reduce.
> > >>>>>
> > >>>>> I think that reduce may not make sense because reduce
> > >>>>>
> > >>>>> assumes
> > >>>>>
> > >>>>> that
> > >>>>>
> > >>>>> the
> > >>>>>
> > >>>>> input values are the same as the output values. With
> > >>>>>
> > >>>>> cogroup
> > >>>>>
> > >>>>> ...
> > >
> > > --
> > > Signature
> > > <http://www.openbet.com/>     Michal Borowiecki
> > > Senior Software Engineer L4
> > >       T:      +44 208 742 1600
> > >
> > >
> > >       +44 203 249 8448
> > >
> > >
> > >
> > >       E:      michal.borowiecki@openbet.com
> > >       W:      www.openbet.com <http://www.openbet.com/>
> > >
> > >
> > >       OpenBet Ltd
> > >
> > >       Chiswick Park Building 9
> > >
> > >       566 Chiswick High Rd
> > >
> > >       London
> > >
> > >       W4 5XT
> > >
> > >       UK
> > >
> > >
> > > <https://www.openbet.com/email_promo>
> > >
> > > This message is confidential and intended only for the addressee. If
> you
> > > have received this message in error, please immediately notify the
> > > postmaster@openbet.com <ma...@openbet.com> and delete it
> > > from your system as well as any copies. The content of e-mails as well
> > > as traffic data may be monitored by OpenBet for employment and security
> > > purposes. To protect the environment please do not print this e-mail
> > > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park
> Building
> > > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> > > registered in England and Wales. Registered no. 3134634. VAT no.
> > > GB927523612
> > >
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Bill Bejeck <bb...@gmail.com>.
+1 on deferring discussion on Serdes until API improvements are ironed out.

On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax <ma...@confluent.io>
wrote:

> Hi,
>
> I am just catching up on this thread. (1) as most people agree, we
> should not add anything to KStreamBuilder (btw: we actually plan to move
> #merge() to KStream and deprecate it on KStreamBuilder as it's a quite
> unnatural API atm).
>
> About specifying Serdes: there is still the idea to improve to overall
> API from the current "we are adding more overloads"-pattern to a
> builder-like pattern. This might make the whole discussion void if we do
> this. Thus, it might make sense to keep this in mind (or even delay this
> KIP?). It seems a waste of time to discuss all this if we are going to
> chance it in 2 month anyway... Just saying.
>
>
> -Matthias
>
> On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> > You're right, I haven't thought of that.
> >
> > Cheers,
> >
> > Michał
> >
> >
> > On 13/06/17 13:00, Kyle Winkelman wrote:
> >> First, I would prefer not calling it aggregate because there are already
> >> plenty of aggregate methods.
> >>
> >> Second, I dont think this would really work because after each aggregate
> >> you now have a unique KTable (someone may want a table with 4 streams
> and
> >> reuse those 4 in another table but with one more stream added) and
> unless
> >> we completely duplicate everything every time this isnt really possible.
> >> Additionally, the cogroup way just requires 1 more call to create two
> >> different tables (normal, windowed, and session windowed) this new way
> >> would require copying the aggregate chain.
> >>
> >> Another way to think about it is with cogroup we know that when they
> call
> >> aggregate they arent going to be adding any more aggregators to that
> table
> >> but your way requires us to assume they are done adding aggregators
> after
> >> each call so we must return a ktable just to possibly not need to have
> >> created it.
> >>
> >> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <
> michal.borowiecki@openbet.com>
> >> wrote:
> >>
> >>> Actually, just had a thought. It started with naming.
> >>>
> >>> Are we actually co-grouping these streams or are we co-aggregating
> them?
> >>>
> >>> After all, in each of the cogroup calls we are providing an Aggregator
> >>> implementation.
> >>>
> >>>
> >>> If they are really co-aggregated, why don't we turn this around:
> >>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").
> groupByKey();
> >>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").
> groupByKey();
> >>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").
> groupByKey();
> >>>
> >>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
> >>> aggValueSerde1) // this is the unchanged aggregate method
> >>>         .aggregate(grouped2, aggregator2)  // this is a new method
> >>>         .aggregate(grouped3, aggregator3); // ditto
> >>>
> >>> This means instead of adding cogroup methods on KGroupStream interface,
> >>> adding aggregate method on KTable interface.
> >>>
> >>> Is that feasible?
> >>>
> >>> Cheers,
> >>> Michał
> >>>
> >>> On 13/06/17 10:56, Michal Borowiecki wrote:
> >>>
> >>> Also, I still feel that putting initializer on the first cogroup can
> >>> mislead users into thinking the first stream is in some way special.
> >>> Just my 5c.
> >>> Michał
> >>>
> >>> On 13/06/17 09:54, Michal Borowiecki wrote:
> >>>
> >>> Agree completely with the argument for serdes belonging in the same
> place
> >>> as the state store name, which is in the aggregate method.
> >>>
> >>> Cheers,
> >>>
> >>> Michał
> >>>
> >>> On 12/06/17 18:20, Xavier Léauté wrote:
> >>>
> >>> I think we are discussing two separate things here, so it might be
> worth
> >>> clarifying:
> >>>
> >>> 1) the position of the initializer with respect to the aggregators. If
> I
> >>> understand correctly, Guozhang seems to think it is more natural to
> specify
> >>> the initializer first, despite it not bearing any relation to the first
> >>> aggregator. I can see the argument for specifying the initializer
> first,
> >>> but I think it is debatable whether mixing it into the first cogroup
> call
> >>> leads to a cleaner API or not.
> >>>
> >>> 2) where the serde should be defined (if necessary). Looking at our
> >>> existing APIs in KGroupedStreams, we always offer two aggregate()
> >>> methods. The first one takes the name of the store and associated
> aggregate
> >>> value serde e.g. KGroupedStream.aggregate(Initializer<VR> initializer,
> >>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR>
> aggValueSerde,
> >>> String queryableStoreName)
> >>> The second one only takes a state store supplier, and does not specify
> any
> >>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> >>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
> >>> StateStoreSupplier<KeyValueStore> storeSupplier)
> >>> Presumably, when specifying a state store supplier it shouldn't be
> >>> necessary to specify an aggregate value serde, since the provided
> >>> statestore might not need to serialize the values (e.g. it may just
> keep
> >>> them as regular objects in heap) or it may have its own
> >>> internal serialization format.
> >>>
> >>> For consistency I think it would be valuable to preserve the same two
> >>> aggregate methods for cogroup as well. Since the serde is only
> required in
> >>> one of the two cases, I believe the serde has no place in the first
> >>> cogroup() call and should only have to be specified as part of the
> >>> aggregate() method that takes a state store name. In the case of a
> state
> >>> store supplier, no serde would be necessary.
> >>>
> >>>
> >>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>>
> >>>> I'd agree that the aggregate value serde and the initializer does not
> >>>> bear direct relationship with the first `cogroup` calls, but after I
> tried
> >>>> to write some example code with these two different set of APIs I
> felt the
> >>>> current APIs just program more naturally.
> >>>>
> >>>> I know it is kinda subjective, but I do think that user experience
> may be
> >>>> more important as a deciding factor than the logical argument for
> public
> >>>> interfaces. So I'd recommend people to also try out writing some
> example
> >>>> lines also and we can circle back and discuss which one feels more
> natural
> >>>> to write code.
> >>>>
> >>>>
> >>>> Guozhang
> >>>>
> >>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> >>>> michal.borowiecki@openbet.com> wrote:
> >>>>
> >>>>> I feel it would make more sense to move the initializer and serde to
> the
> >>>>> final aggregate statement, since the serde only applies to the state
> >>>>> store,
> >>>>> and the initializer doesn't bear any relation to the first group in
> >>>>> particular.
> >>>>>
> >>>>> +1 for moving initializer and serde from cogroup() to the aggregate()
> >>>>> for the reasons mentioned above.
> >>>>>
> >>>>> Cheers,
> >>>>>
> >>>>> Michał
> >>>>>
> >>>>> On 08/06/17 22:44, Guozhang Wang wrote:
> >>>>>
> >>>> Note that although the internal `AbstractStoreSupplier` does maintain
> the
> >>>>> key-value serdes, we do not enforce the interface of
> `StateStoreSupplier`
> >>>>> to always retain that information, and hence we cannot assume that
> >>>>> StateStoreSuppliers always retain key / value serdes.
> >>>>>
> >>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io>
> <xa...@confluent.io> wrote:
> >>>>>
> >>>>>
> >>>>> Another reason for the serde not to be in the first cogroup call, is
> that
> >>>>> the serde should not be required if you pass a StateStoreSupplier to
> >>>>> aggregate()
> >>>>>
> >>>>> Regarding the aggregated type <T> I don't the why initializer should
> be
> >>>>> favored over aggregator to define the type. In my mind separating the
> >>>>> initializer into the last aggregate call clearly indicates that the
> >>>>> initializer is independent of any of the aggregators or streams and
> that we
> >>>>> don't wait for grouped1 events to initialize the co-group.
> >>>>>
> >>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <
> wangguoz@gmail.com> wrote:
> >>>>>
> >>>>>
> >>>>> On a second thought... This is the current proposal API
> >>>>>
> >>>>>
> >>>>> ```
> >>>>>
> >>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
> >>>>>
> >>>>> final
> >>>>>
> >>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> >>>>> aggValueSerde)
> >>>>>
> >>>>> ```
> >>>>>
> >>>>>
> >>>>> If we do not have the initializer in the first co-group it might be
> a bit
> >>>>> awkward for users to specify the aggregator that returns a typed <T>
> >>>>>
> >>>>> value?
> >>>>>
> >>>>> Maybe it is still better to put these two functions in the same api?
> >>>>>
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com>
> <wa...@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> This suggestion lgtm. I would vote for the first alternative than
> >>>>>
> >>>>> adding
> >>>>>
> >>>>> it to the `KStreamBuilder` though.
> >>>>>
> >>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
> <xa...@confluent.io>
> >>>>> wrote:
> >>>>>
> >>>>>
> >>>>> I have a minor suggestion to make the API a little bit more
> symmetric.
> >>>>> I feel it would make more sense to move the initializer and serde to
> >>>>>
> >>>>> the
> >>>>>
> >>>>> final aggregate statement, since the serde only applies to the state
> >>>>> store,
> >>>>> and the initializer doesn't bear any relation to the first group in
> >>>>> particular. It would end up looking like this:
> >>>>>
> >>>>> KTable<K, CG> cogrouped =
> >>>>>     grouped1.cogroup(aggregator1)
> >>>>>             .cogroup(grouped2, aggregator2)
> >>>>>             .cogroup(grouped3, aggregator3)
> >>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
> >>>>>
> >>>>> Alternatively, we could move the the first cogroup() method to
> >>>>> KStreamBuilder, similar to how we have .merge()
> >>>>> and end up with an api that would be even more symmetric.
> >>>>>
> >>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
> >>>>>               .cogroup(grouped2, aggregator2)
> >>>>>               .cogroup(grouped3, aggregator3)
> >>>>>               .aggregate(initializer1, aggValueSerde, storeName1);
> >>>>>
> >>>>> This doesn't have to be a blocker, but I thought it would make the
> API
> >>>>> just
> >>>>> a tad cleaner.
> >>>>>
> >>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <
> wangguoz@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> Kyle,
> >>>>>
> >>>>> Thanks a lot for the updated KIP. It looks good to me.
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <
> jim@jagunet.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> This makes much more sense to me. +1
> >>>>>
> >>>>>
> >>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> >>>>>
> >>>>> winkelman.kyle@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> I have updated the KIP and my PR. Let me know what you think.
> >>>>> To created a cogrouped stream just call cogroup on a
> >>>>>
> >>>>> KgroupedStream
> >>>>>
> >>>>> and
> >>>>>
> >>>>> supply the initializer, aggValueSerde, and an aggregator. Then
> >>>>>
> >>>>> continue
> >>>>>
> >>>>> adding kgroupedstreams and aggregators. Then call one of the
> >>>>>
> >>>>> many
> >>>>>
> >>>>> aggregate
> >>>>>
> >>>>> calls to create a KTable.
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <
> damian.guy@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> Hi Kyle,
> >>>>>
> >>>>> Thanks for the update. I think just one initializer makes sense
> >>>>>
> >>>>> as
> >>>>>
> >>>>> it
> >>>>>
> >>>>> should only be called once per key and generally it is just
> >>>>>
> >>>>> going
> >>>>>
> >>>>> to
> >>>>>
> >>>>> create
> >>>>>
> >>>>> a new instance of whatever the Aggregate class is.
> >>>>>
> >>>>> Cheers,
> >>>>> Damian
> >>>>>
> >>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> >>>>>
> >>>>> winkelman.kyle@gmail.com
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>>
> >>>>> Hello all,
> >>>>>
> >>>>> I have spent some more time on this and the best alternative I
> >>>>>
> >>>>> have
> >>>>>
> >>>>> come
> >>>>>
> >>>>> up
> >>>>>
> >>>>> with is:
> >>>>> KGroupedStream has a single cogroup call that takes an
> >>>>>
> >>>>> initializer
> >>>>>
> >>>>> and
> >>>>>
> >>>>> an
> >>>>>
> >>>>> aggregator.
> >>>>> CogroupedKStream has a cogroup call that takes additional
> >>>>>
> >>>>> groupedStream
> >>>>>
> >>>>> aggregator pairs.
> >>>>> CogroupedKStream has multiple aggregate methods that create
> >>>>>
> >>>>> the
> >>>>>
> >>>>> different
> >>>>>
> >>>>> stores.
> >>>>>
> >>>>> I plan on updating the kip but I want people's input on if we
> >>>>>
> >>>>> should
> >>>>>
> >>>>> have
> >>>>>
> >>>>> the initializer be passed in once at the beginning or if we
> >>>>>
> >>>>> should
> >>>>>
> >>>>> instead
> >>>>>
> >>>>> have the initializer be required for each call to one of the
> >>>>>
> >>>>> aggregate
> >>>>>
> >>>>> calls. The first makes more sense to me but doesnt allow the
> >>>>>
> >>>>> user
> >>>>>
> >>>>> to
> >>>>>
> >>>>> specify different initializers for different tables.
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> >>>>>
> >>>>> winkelman.kyle@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>>
> >>>>> Yea I really like that idea I'll see what I can do to update
> >>>>>
> >>>>> the
> >>>>>
> >>>>> kip
> >>>>>
> >>>>> and
> >>>>>
> >>>>> my pr when I have some time. I'm not sure how well creating
> >>>>>
> >>>>> the
> >>>>>
> >>>>> kstreamaggregates will go though because at that point I will
> >>>>>
> >>>>> have
> >>>>>
> >>>>> thrown
> >>>>>
> >>>>> away the type of the values. It will be type safe I just may
> >>>>>
> >>>>> need to
> >>>>>
> >>>>> do a
> >>>>>
> >>>>> little forcing.
> >>>>>
> >>>>> Thanks,
> >>>>> Kyle
> >>>>>
> >>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> Kyle,
> >>>>>
> >>>>> Thanks for the explanations, my previous read on the wiki
> >>>>>
> >>>>> examples
> >>>>>
> >>>>> was
> >>>>>
> >>>>> wrong.
> >>>>>
> >>>>> So I guess my motivation should be "reduced" to: can we move
> >>>>>
> >>>>> the
> >>>>>
> >>>>> window
> >>>>>
> >>>>> specs param from "KGroupedStream#cogroup(..)" to
> >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> >>>>>
> >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> >>>>>
> >>>>> from
> >>>>>
> >>>>> 3
> >>>>>
> >>>>> to
> >>>>>
> >>>>> 2.
> >>>>>
> >>>>> 2. major: this is for extensibility of the APIs, and since
> >>>>>
> >>>>> we
> >>>>>
> >>>>> are
> >>>>>
> >>>>> removing
> >>>>>
> >>>>> the "Evolving" annotations on Streams it may be harder to
> >>>>>
> >>>>> change it
> >>>>>
> >>>>> again
> >>>>>
> >>>>> in the future. The extended use cases are that people wanted
> >>>>>
> >>>>> to
> >>>>>
> >>>>> have
> >>>>>
> >>>>> windowed running aggregates on different granularities, e.g.
> >>>>>
> >>>>> "give
> >>>>>
> >>>>> me
> >>>>>
> >>>>> the
> >>>>>
> >>>>> counts per-minute, per-hour, per-day and per-week", and
> >>>>>
> >>>>> today
> >>>>>
> >>>>> in
> >>>>>
> >>>>> DSL
> >>>>>
> >>>>> we
> >>>>>
> >>>>> need to specify that case in multiple aggregate operators,
> >>>>>
> >>>>> which
> >>>>>
> >>>>> gets
> >>>>>
> >>>>> a
> >>>>>
> >>>>> state store / changelog, etc. And it is possible to optimize
> >>>>>
> >>>>> it
> >>>>>
> >>>>> as
> >>>>>
> >>>>> well
> >>>>>
> >>>>> to
> >>>>>
> >>>>> a single state store. Its implementation would be tricky as
> >>>>>
> >>>>> you
> >>>>>
> >>>>> need
> >>>>>
> >>>>> to
> >>>>>
> >>>>> contain different lengthed windows within your window store
> >>>>>
> >>>>> but
> >>>>>
> >>>>> just
> >>>>>
> >>>>> from
> >>>>>
> >>>>> the public API point of view, it could be specified as:
> >>>>>
> >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >>>>> "state-store-name");
> >>>>>
> >>>>> table1 = stream.aggregate(/*per-minute window*/)
> >>>>> table2 = stream.aggregate(/*per-hour window*/)
> >>>>> table3 = stream.aggregate(/*per-day window*/)
> >>>>>
> >>>>> while underlying we are only using a single store
> >>>>>
> >>>>> "state-store-name"
> >>>>>
> >>>>> for
> >>>>>
> >>>>> it.
> >>>>>
> >>>>>
> >>>>> Although this feature is out of the scope of this KIP, I'd
> >>>>>
> >>>>> like
> >>>>>
> >>>>> to
> >>>>>
> >>>>> discuss
> >>>>>
> >>>>> if we can "leave the door open" to make such changes without
> >>>>>
> >>>>> modifying
> >>>>>
> >>>>> the
> >>>>>
> >>>>> public APIs .
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> >>>>>
> >>>>> winkelman.kyle@gmail.com
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>>
> >>>>> I allow defining a single window/sessionwindow one time
> >>>>>
> >>>>> when
> >>>>>
> >>>>> you
> >>>>>
> >>>>> make
> >>>>>
> >>>>> the
> >>>>>
> >>>>> cogroup call from a KGroupedStream. From then on you are
> >>>>>
> >>>>> using
> >>>>>
> >>>>> the
> >>>>>
> >>>>> cogroup
> >>>>>
> >>>>> call from with in CogroupedKStream which doesnt accept any
> >>>>>
> >>>>> additional
> >>>>>
> >>>>> windows/sessionwindows.
> >>>>>
> >>>>> Is this what you meant by your question or did I
> >>>>>
> >>>>> misunderstand?
> >>>>>
> >>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> >>>>>
> >>>>> wangguoz@gmail.com
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> Another question that came to me is on "window alignment":
> >>>>>
> >>>>> from
> >>>>>
> >>>>> the
> >>>>>
> >>>>> KIP
> >>>>>
> >>>>> it
> >>>>>
> >>>>> seems you are allowing users to specify a (potentially
> >>>>>
> >>>>> different)
> >>>>>
> >>>>> window
> >>>>>
> >>>>> spec in each co-grouped input stream. So if these window
> >>>>>
> >>>>> specs
> >>>>>
> >>>>> are
> >>>>>
> >>>>> different how should we "align" them with different input
> >>>>>
> >>>>> streams? I
> >>>>>
> >>>>> think
> >>>>>
> >>>>> it is more natural to only specify on window spec in the
> >>>>>
> >>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >>>>>
> >>>>>
> >>>>> And remove it from the cogroup() functions. WDYT?
> >>>>>
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> >>>>>
> >>>>> wangguoz@gmail.com>
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> Thanks for the proposal Kyle, this is a quite common use
> >>>>>
> >>>>> case
> >>>>>
> >>>>> to
> >>>>>
> >>>>> support
> >>>>>
> >>>>> such multi-way table join (i.e. N source tables with N
> >>>>>
> >>>>> aggregate
> >>>>>
> >>>>> func)
> >>>>>
> >>>>> with
> >>>>>
> >>>>> a single store and N+1 serdes, I have seen lots of people
> >>>>>
> >>>>> using
> >>>>>
> >>>>> the
> >>>>>
> >>>>> low-level PAPI to achieve this goal.
> >>>>>
> >>>>>
> >>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >>>>>
> >>>>> winkelman.kyle@gmail.com
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>> I like your point about not handling other cases such as
> >>>>>
> >>>>> count
> >>>>>
> >>>>> and
> >>>>>
> >>>>> reduce.
> >>>>>
> >>>>> I think that reduce may not make sense because reduce
> >>>>>
> >>>>> assumes
> >>>>>
> >>>>> that
> >>>>>
> >>>>> the
> >>>>>
> >>>>> input values are the same as the output values. With
> >>>>>
> >>>>> cogroup
> >>>>>
> >>>>> ...
> >
> > --
> > Signature
> > <http://www.openbet.com/>     Michal Borowiecki
> > Senior Software Engineer L4
> >       T:      +44 208 742 1600
> >
> >
> >       +44 203 249 8448
> >
> >
> >
> >       E:      michal.borowiecki@openbet.com
> >       W:      www.openbet.com <http://www.openbet.com/>
> >
> >
> >       OpenBet Ltd
> >
> >       Chiswick Park Building 9
> >
> >       566 Chiswick High Rd
> >
> >       London
> >
> >       W4 5XT
> >
> >       UK
> >
> >
> > <https://www.openbet.com/email_promo>
> >
> > This message is confidential and intended only for the addressee. If you
> > have received this message in error, please immediately notify the
> > postmaster@openbet.com <ma...@openbet.com> and delete it
> > from your system as well as any copies. The content of e-mails as well
> > as traffic data may be monitored by OpenBet for employment and security
> > purposes. To protect the environment please do not print this e-mail
> > unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> > 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> > registered in England and Wales. Registered no. 3134634. VAT no.
> > GB927523612
> >
>
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

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

I am just catching up on this thread. (1) as most people agree, we
should not add anything to KStreamBuilder (btw: we actually plan to move
#merge() to KStream and deprecate it on KStreamBuilder as it's a quite
unnatural API atm).

About specifying Serdes: there is still the idea to improve to overall
API from the current "we are adding more overloads"-pattern to a
builder-like pattern. This might make the whole discussion void if we do
this. Thus, it might make sense to keep this in mind (or even delay this
KIP?). It seems a waste of time to discuss all this if we are going to
chance it in 2 month anyway... Just saying.


-Matthias

On 6/13/17 8:05 AM, Michal Borowiecki wrote:
> You're right, I haven't thought of that.
> 
> Cheers,
> 
> Michał
> 
> 
> On 13/06/17 13:00, Kyle Winkelman wrote:
>> First, I would prefer not calling it aggregate because there are already
>> plenty of aggregate methods.
>>
>> Second, I dont think this would really work because after each aggregate
>> you now have a unique KTable (someone may want a table with 4 streams and
>> reuse those 4 in another table but with one more stream added) and unless
>> we completely duplicate everything every time this isnt really possible.
>> Additionally, the cogroup way just requires 1 more call to create two
>> different tables (normal, windowed, and session windowed) this new way
>> would require copying the aggregate chain.
>>
>> Another way to think about it is with cogroup we know that when they call
>> aggregate they arent going to be adding any more aggregators to that table
>> but your way requires us to assume they are done adding aggregators after
>> each call so we must return a ktable just to possibly not need to have
>> created it.
>>
>> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <mi...@openbet.com>
>> wrote:
>>
>>> Actually, just had a thought. It started with naming.
>>>
>>> Are we actually co-grouping these streams or are we co-aggregating them?
>>>
>>> After all, in each of the cogroup calls we are providing an Aggregator
>>> implementation.
>>>
>>>
>>> If they are really co-aggregated, why don't we turn this around:
>>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").groupByKey();
>>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").groupByKey();
>>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").groupByKey();
>>>
>>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
>>> aggValueSerde1) // this is the unchanged aggregate method
>>>         .aggregate(grouped2, aggregator2)  // this is a new method
>>>         .aggregate(grouped3, aggregator3); // ditto
>>>
>>> This means instead of adding cogroup methods on KGroupStream interface,
>>> adding aggregate method on KTable interface.
>>>
>>> Is that feasible?
>>>
>>> Cheers,
>>> Michał
>>>
>>> On 13/06/17 10:56, Michal Borowiecki wrote:
>>>
>>> Also, I still feel that putting initializer on the first cogroup can
>>> mislead users into thinking the first stream is in some way special.
>>> Just my 5c.
>>> Michał
>>>
>>> On 13/06/17 09:54, Michal Borowiecki wrote:
>>>
>>> Agree completely with the argument for serdes belonging in the same place
>>> as the state store name, which is in the aggregate method.
>>>
>>> Cheers,
>>>
>>> Michał
>>>
>>> On 12/06/17 18:20, Xavier Léauté wrote:
>>>
>>> I think we are discussing two separate things here, so it might be worth
>>> clarifying:
>>>
>>> 1) the position of the initializer with respect to the aggregators. If I
>>> understand correctly, Guozhang seems to think it is more natural to specify
>>> the initializer first, despite it not bearing any relation to the first
>>> aggregator. I can see the argument for specifying the initializer first,
>>> but I think it is debatable whether mixing it into the first cogroup call
>>> leads to a cleaner API or not.
>>>
>>> 2) where the serde should be defined (if necessary). Looking at our
>>> existing APIs in KGroupedStreams, we always offer two aggregate()
>>> methods. The first one takes the name of the store and associated aggregate
>>> value serde e.g. KGroupedStream.aggregate(Initializer<VR> initializer,
>>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR> aggValueSerde,
>>> String queryableStoreName)
>>> The second one only takes a state store supplier, and does not specify any
>>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
>>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
>>> StateStoreSupplier<KeyValueStore> storeSupplier)
>>> Presumably, when specifying a state store supplier it shouldn't be
>>> necessary to specify an aggregate value serde, since the provided
>>> statestore might not need to serialize the values (e.g. it may just keep
>>> them as regular objects in heap) or it may have its own
>>> internal serialization format.
>>>
>>> For consistency I think it would be valuable to preserve the same two
>>> aggregate methods for cogroup as well. Since the serde is only required in
>>> one of the two cases, I believe the serde has no place in the first
>>> cogroup() call and should only have to be specified as part of the
>>> aggregate() method that takes a state store name. In the case of a state
>>> store supplier, no serde would be necessary.
>>>
>>>
>>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com> wrote:
>>>
>>>> I'd agree that the aggregate value serde and the initializer does not
>>>> bear direct relationship with the first `cogroup` calls, but after I tried
>>>> to write some example code with these two different set of APIs I felt the
>>>> current APIs just program more naturally.
>>>>
>>>> I know it is kinda subjective, but I do think that user experience may be
>>>> more important as a deciding factor than the logical argument for public
>>>> interfaces. So I'd recommend people to also try out writing some example
>>>> lines also and we can circle back and discuss which one feels more natural
>>>> to write code.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
>>>> michal.borowiecki@openbet.com> wrote:
>>>>
>>>>> I feel it would make more sense to move the initializer and serde to the
>>>>> final aggregate statement, since the serde only applies to the state
>>>>> store,
>>>>> and the initializer doesn't bear any relation to the first group in
>>>>> particular.
>>>>>
>>>>> +1 for moving initializer and serde from cogroup() to the aggregate()
>>>>> for the reasons mentioned above.
>>>>>
>>>>> Cheers,
>>>>>
>>>>> Michał
>>>>>
>>>>> On 08/06/17 22:44, Guozhang Wang wrote:
>>>>>
>>>> Note that although the internal `AbstractStoreSupplier` does maintain the
>>>>> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>>>> to always retain that information, and hence we cannot assume that
>>>>> StateStoreSuppliers always retain key / value serdes.
>>>>>
>>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io> wrote:
>>>>>
>>>>>
>>>>> Another reason for the serde not to be in the first cogroup call, is that
>>>>> the serde should not be required if you pass a StateStoreSupplier to
>>>>> aggregate()
>>>>>
>>>>> Regarding the aggregated type <T> I don't the why initializer should be
>>>>> favored over aggregator to define the type. In my mind separating the
>>>>> initializer into the last aggregate call clearly indicates that the
>>>>> initializer is independent of any of the aggregators or streams and that we
>>>>> don't wait for grouped1 events to initialize the co-group.
>>>>>
>>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com> wrote:
>>>>>
>>>>>
>>>>> On a second thought... This is the current proposal API
>>>>>
>>>>>
>>>>> ```
>>>>>
>>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>>>
>>>>> final
>>>>>
>>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>>>> aggValueSerde)
>>>>>
>>>>> ```
>>>>>
>>>>>
>>>>> If we do not have the initializer in the first co-group it might be a bit
>>>>> awkward for users to specify the aggregator that returns a typed <T>
>>>>>
>>>>> value?
>>>>>
>>>>> Maybe it is still better to put these two functions in the same api?
>>>>>
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> This suggestion lgtm. I would vote for the first alternative than
>>>>>
>>>>> adding
>>>>>
>>>>> it to the `KStreamBuilder` though.
>>>>>
>>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io>
>>>>> wrote:
>>>>>
>>>>>
>>>>> I have a minor suggestion to make the API a little bit more symmetric.
>>>>> I feel it would make more sense to move the initializer and serde to
>>>>>
>>>>> the
>>>>>
>>>>> final aggregate statement, since the serde only applies to the state
>>>>> store,
>>>>> and the initializer doesn't bear any relation to the first group in
>>>>> particular. It would end up looking like this:
>>>>>
>>>>> KTable<K, CG> cogrouped =
>>>>>     grouped1.cogroup(aggregator1)
>>>>>             .cogroup(grouped2, aggregator2)
>>>>>             .cogroup(grouped3, aggregator3)
>>>>>             .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>
>>>>> Alternatively, we could move the the first cogroup() method to
>>>>> KStreamBuilder, similar to how we have .merge()
>>>>> and end up with an api that would be even more symmetric.
>>>>>
>>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>>               .cogroup(grouped2, aggregator2)
>>>>>               .cogroup(grouped3, aggregator3)
>>>>>               .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>
>>>>> This doesn't have to be a blocker, but I thought it would make the API
>>>>> just
>>>>> a tad cleaner.
>>>>>
>>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> Kyle,
>>>>>
>>>>> Thanks a lot for the updated KIP. It looks good to me.
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <ji...@jagunet.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> This makes much more sense to me. +1
>>>>>
>>>>>
>>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>>
>>>>> winkelman.kyle@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> I have updated the KIP and my PR. Let me know what you think.
>>>>> To created a cogrouped stream just call cogroup on a
>>>>>
>>>>> KgroupedStream
>>>>>
>>>>> and
>>>>>
>>>>> supply the initializer, aggValueSerde, and an aggregator. Then
>>>>>
>>>>> continue
>>>>>
>>>>> adding kgroupedstreams and aggregators. Then call one of the
>>>>>
>>>>> many
>>>>>
>>>>> aggregate
>>>>>
>>>>> calls to create a KTable.
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <da...@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> Hi Kyle,
>>>>>
>>>>> Thanks for the update. I think just one initializer makes sense
>>>>>
>>>>> as
>>>>>
>>>>> it
>>>>>
>>>>> should only be called once per key and generally it is just
>>>>>
>>>>> going
>>>>>
>>>>> to
>>>>>
>>>>> create
>>>>>
>>>>> a new instance of whatever the Aggregate class is.
>>>>>
>>>>> Cheers,
>>>>> Damian
>>>>>
>>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>>
>>>>> winkelman.kyle@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>>
>>>>> Hello all,
>>>>>
>>>>> I have spent some more time on this and the best alternative I
>>>>>
>>>>> have
>>>>>
>>>>> come
>>>>>
>>>>> up
>>>>>
>>>>> with is:
>>>>> KGroupedStream has a single cogroup call that takes an
>>>>>
>>>>> initializer
>>>>>
>>>>> and
>>>>>
>>>>> an
>>>>>
>>>>> aggregator.
>>>>> CogroupedKStream has a cogroup call that takes additional
>>>>>
>>>>> groupedStream
>>>>>
>>>>> aggregator pairs.
>>>>> CogroupedKStream has multiple aggregate methods that create
>>>>>
>>>>> the
>>>>>
>>>>> different
>>>>>
>>>>> stores.
>>>>>
>>>>> I plan on updating the kip but I want people's input on if we
>>>>>
>>>>> should
>>>>>
>>>>> have
>>>>>
>>>>> the initializer be passed in once at the beginning or if we
>>>>>
>>>>> should
>>>>>
>>>>> instead
>>>>>
>>>>> have the initializer be required for each call to one of the
>>>>>
>>>>> aggregate
>>>>>
>>>>> calls. The first makes more sense to me but doesnt allow the
>>>>>
>>>>> user
>>>>>
>>>>> to
>>>>>
>>>>> specify different initializers for different tables.
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>>
>>>>> winkelman.kyle@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>>
>>>>> Yea I really like that idea I'll see what I can do to update
>>>>>
>>>>> the
>>>>>
>>>>> kip
>>>>>
>>>>> and
>>>>>
>>>>> my pr when I have some time. I'm not sure how well creating
>>>>>
>>>>> the
>>>>>
>>>>> kstreamaggregates will go though because at that point I will
>>>>>
>>>>> have
>>>>>
>>>>> thrown
>>>>>
>>>>> away the type of the values. It will be type safe I just may
>>>>>
>>>>> need to
>>>>>
>>>>> do a
>>>>>
>>>>> little forcing.
>>>>>
>>>>> Thanks,
>>>>> Kyle
>>>>>
>>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>> Kyle,
>>>>>
>>>>> Thanks for the explanations, my previous read on the wiki
>>>>>
>>>>> examples
>>>>>
>>>>> was
>>>>>
>>>>> wrong.
>>>>>
>>>>> So I guess my motivation should be "reduced" to: can we move
>>>>>
>>>>> the
>>>>>
>>>>> window
>>>>>
>>>>> specs param from "KGroupedStream#cogroup(..)" to
>>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>>
>>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>>>>>
>>>>> from
>>>>>
>>>>> 3
>>>>>
>>>>> to
>>>>>
>>>>> 2.
>>>>>
>>>>> 2. major: this is for extensibility of the APIs, and since
>>>>>
>>>>> we
>>>>>
>>>>> are
>>>>>
>>>>> removing
>>>>>
>>>>> the "Evolving" annotations on Streams it may be harder to
>>>>>
>>>>> change it
>>>>>
>>>>> again
>>>>>
>>>>> in the future. The extended use cases are that people wanted
>>>>>
>>>>> to
>>>>>
>>>>> have
>>>>>
>>>>> windowed running aggregates on different granularities, e.g.
>>>>>
>>>>> "give
>>>>>
>>>>> me
>>>>>
>>>>> the
>>>>>
>>>>> counts per-minute, per-hour, per-day and per-week", and
>>>>>
>>>>> today
>>>>>
>>>>> in
>>>>>
>>>>> DSL
>>>>>
>>>>> we
>>>>>
>>>>> need to specify that case in multiple aggregate operators,
>>>>>
>>>>> which
>>>>>
>>>>> gets
>>>>>
>>>>> a
>>>>>
>>>>> state store / changelog, etc. And it is possible to optimize
>>>>>
>>>>> it
>>>>>
>>>>> as
>>>>>
>>>>> well
>>>>>
>>>>> to
>>>>>
>>>>> a single state store. Its implementation would be tricky as
>>>>>
>>>>> you
>>>>>
>>>>> need
>>>>>
>>>>> to
>>>>>
>>>>> contain different lengthed windows within your window store
>>>>>
>>>>> but
>>>>>
>>>>> just
>>>>>
>>>>> from
>>>>>
>>>>> the public API point of view, it could be specified as:
>>>>>
>>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>> "state-store-name");
>>>>>
>>>>> table1 = stream.aggregate(/*per-minute window*/)
>>>>> table2 = stream.aggregate(/*per-hour window*/)
>>>>> table3 = stream.aggregate(/*per-day window*/)
>>>>>
>>>>> while underlying we are only using a single store
>>>>>
>>>>> "state-store-name"
>>>>>
>>>>> for
>>>>>
>>>>> it.
>>>>>
>>>>>
>>>>> Although this feature is out of the scope of this KIP, I'd
>>>>>
>>>>> like
>>>>>
>>>>> to
>>>>>
>>>>> discuss
>>>>>
>>>>> if we can "leave the door open" to make such changes without
>>>>>
>>>>> modifying
>>>>>
>>>>> the
>>>>>
>>>>> public APIs .
>>>>>
>>>>> Guozhang
>>>>>
>>>>>
>>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>>
>>>>> winkelman.kyle@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>>
>>>>> I allow defining a single window/sessionwindow one time
>>>>>
>>>>> when
>>>>>
>>>>> you
>>>>>
>>>>> make
>>>>>
>>>>> the
>>>>>
>>>>> cogroup call from a KGroupedStream. From then on you are
>>>>>
>>>>> using
>>>>>
>>>>> the
>>>>>
>>>>> cogroup
>>>>>
>>>>> call from with in CogroupedKStream which doesnt accept any
>>>>>
>>>>> additional
>>>>>
>>>>> windows/sessionwindows.
>>>>>
>>>>> Is this what you meant by your question or did I
>>>>>
>>>>> misunderstand?
>>>>>
>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>>>
>>>>> wangguoz@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>> Another question that came to me is on "window alignment":
>>>>>
>>>>> from
>>>>>
>>>>> the
>>>>>
>>>>> KIP
>>>>>
>>>>> it
>>>>>
>>>>> seems you are allowing users to specify a (potentially
>>>>>
>>>>> different)
>>>>>
>>>>> window
>>>>>
>>>>> spec in each co-grouped input stream. So if these window
>>>>>
>>>>> specs
>>>>>
>>>>> are
>>>>>
>>>>> different how should we "align" them with different input
>>>>>
>>>>> streams? I
>>>>>
>>>>> think
>>>>>
>>>>> it is more natural to only specify on window spec in the
>>>>>
>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>
>>>>>
>>>>> And remove it from the cogroup() functions. WDYT?
>>>>>
>>>>>
>>>>> Guozhang
>>>>>
>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>>
>>>>> wangguoz@gmail.com>
>>>>>
>>>>> wrote:
>>>>>
>>>>> Thanks for the proposal Kyle, this is a quite common use
>>>>>
>>>>> case
>>>>>
>>>>> to
>>>>>
>>>>> support
>>>>>
>>>>> such multi-way table join (i.e. N source tables with N
>>>>>
>>>>> aggregate
>>>>>
>>>>> func)
>>>>>
>>>>> with
>>>>>
>>>>> a single store and N+1 serdes, I have seen lots of people
>>>>>
>>>>> using
>>>>>
>>>>> the
>>>>>
>>>>> low-level PAPI to achieve this goal.
>>>>>
>>>>>
>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>
>>>>> winkelman.kyle@gmail.com
>>>>>
>>>>> wrote:
>>>>>
>>>>> I like your point about not handling other cases such as
>>>>>
>>>>> count
>>>>>
>>>>> and
>>>>>
>>>>> reduce.
>>>>>
>>>>> I think that reduce may not make sense because reduce
>>>>>
>>>>> assumes
>>>>>
>>>>> that
>>>>>
>>>>> the
>>>>>
>>>>> input values are the same as the output values. With
>>>>>
>>>>> cogroup
>>>>>
>>>>> ...
> 
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
> 
> 	
> 	+44 203 249 8448
> 
> 	
> 	 
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
> 
> 	
> 	OpenBet Ltd
> 
> 	Chiswick Park Building 9
> 
> 	566 Chiswick High Rd
> 
> 	London
> 
> 	W4 5XT
> 
> 	UK
> 
> 	
> <https://www.openbet.com/email_promo>
> 
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com <ma...@openbet.com> and delete it
> from your system as well as any copies. The content of e-mails as well
> as traffic data may be monitored by OpenBet for employment and security
> purposes. To protect the environment please do not print this e-mail
> unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building
> 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company
> registered in England and Wales. Registered no. 3134634. VAT no.
> GB927523612
> 


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Michal Borowiecki <mi...@openbet.com>.
You're right, I haven't thought of that.

Cheers,

Michał


On 13/06/17 13:00, Kyle Winkelman wrote:
> First, I would prefer not calling it aggregate because there are already
> plenty of aggregate methods.
>
> Second, I dont think this would really work because after each aggregate
> you now have a unique KTable (someone may want a table with 4 streams and
> reuse those 4 in another table but with one more stream added) and unless
> we completely duplicate everything every time this isnt really possible.
> Additionally, the cogroup way just requires 1 more call to create two
> different tables (normal, windowed, and session windowed) this new way
> would require copying the aggregate chain.
>
> Another way to think about it is with cogroup we know that when they call
> aggregate they arent going to be adding any more aggregators to that table
> but your way requires us to assume they are done adding aggregators after
> each call so we must return a ktable just to possibly not need to have
> created it.
>
> On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <mi...@openbet.com>
> wrote:
>
>> Actually, just had a thought. It started with naming.
>>
>> Are we actually co-grouping these streams or are we co-aggregating them?
>>
>> After all, in each of the cogroup calls we are providing an Aggregator
>> implementation.
>>
>>
>> If they are really co-aggregated, why don't we turn this around:
>> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").groupByKey();
>> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").groupByKey();
>> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").groupByKey();
>>
>> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
>> aggValueSerde1) // this is the unchanged aggregate method
>>          .aggregate(grouped2, aggregator2)  // this is a new method
>>          .aggregate(grouped3, aggregator3); // ditto
>>
>> This means instead of adding cogroup methods on KGroupStream interface,
>> adding aggregate method on KTable interface.
>>
>> Is that feasible?
>>
>> Cheers,
>> Michał
>>
>> On 13/06/17 10:56, Michal Borowiecki wrote:
>>
>> Also, I still feel that putting initializer on the first cogroup can
>> mislead users into thinking the first stream is in some way special.
>> Just my 5c.
>> Michał
>>
>> On 13/06/17 09:54, Michal Borowiecki wrote:
>>
>> Agree completely with the argument for serdes belonging in the same place
>> as the state store name, which is in the aggregate method.
>>
>> Cheers,
>>
>> Michał
>>
>> On 12/06/17 18:20, Xavier Léauté wrote:
>>
>> I think we are discussing two separate things here, so it might be worth
>> clarifying:
>>
>> 1) the position of the initializer with respect to the aggregators. If I
>> understand correctly, Guozhang seems to think it is more natural to specify
>> the initializer first, despite it not bearing any relation to the first
>> aggregator. I can see the argument for specifying the initializer first,
>> but I think it is debatable whether mixing it into the first cogroup call
>> leads to a cleaner API or not.
>>
>> 2) where the serde should be defined (if necessary). Looking at our
>> existing APIs in KGroupedStreams, we always offer two aggregate()
>> methods. The first one takes the name of the store and associated aggregate
>> value serde e.g. KGroupedStream.aggregate(Initializer<VR> initializer,
>> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR> aggValueSerde,
>> String queryableStoreName)
>> The second one only takes a state store supplier, and does not specify any
>> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
>> StateStoreSupplier<KeyValueStore> storeSupplier)
>> Presumably, when specifying a state store supplier it shouldn't be
>> necessary to specify an aggregate value serde, since the provided
>> statestore might not need to serialize the values (e.g. it may just keep
>> them as regular objects in heap) or it may have its own
>> internal serialization format.
>>
>> For consistency I think it would be valuable to preserve the same two
>> aggregate methods for cogroup as well. Since the serde is only required in
>> one of the two cases, I believe the serde has no place in the first
>> cogroup() call and should only have to be specified as part of the
>> aggregate() method that takes a state store name. In the case of a state
>> store supplier, no serde would be necessary.
>>
>>
>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> I'd agree that the aggregate value serde and the initializer does not
>>> bear direct relationship with the first `cogroup` calls, but after I tried
>>> to write some example code with these two different set of APIs I felt the
>>> current APIs just program more naturally.
>>>
>>> I know it is kinda subjective, but I do think that user experience may be
>>> more important as a deciding factor than the logical argument for public
>>> interfaces. So I'd recommend people to also try out writing some example
>>> lines also and we can circle back and discuss which one feels more natural
>>> to write code.
>>>
>>>
>>> Guozhang
>>>
>>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
>>> michal.borowiecki@openbet.com> wrote:
>>>
>>>> I feel it would make more sense to move the initializer and serde to the
>>>> final aggregate statement, since the serde only applies to the state
>>>> store,
>>>> and the initializer doesn't bear any relation to the first group in
>>>> particular.
>>>>
>>>> +1 for moving initializer and serde from cogroup() to the aggregate()
>>>> for the reasons mentioned above.
>>>>
>>>> Cheers,
>>>>
>>>> Michał
>>>>
>>>> On 08/06/17 22:44, Guozhang Wang wrote:
>>>>
>>> Note that although the internal `AbstractStoreSupplier` does maintain the
>>>> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>>> to always retain that information, and hence we cannot assume that
>>>> StateStoreSuppliers always retain key / value serdes.
>>>>
>>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io> wrote:
>>>>
>>>>
>>>> Another reason for the serde not to be in the first cogroup call, is that
>>>> the serde should not be required if you pass a StateStoreSupplier to
>>>> aggregate()
>>>>
>>>> Regarding the aggregated type <T> I don't the why initializer should be
>>>> favored over aggregator to define the type. In my mind separating the
>>>> initializer into the last aggregate call clearly indicates that the
>>>> initializer is independent of any of the aggregators or streams and that we
>>>> don't wait for grouped1 events to initialize the co-group.
>>>>
>>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com> wrote:
>>>>
>>>>
>>>> On a second thought... This is the current proposal API
>>>>
>>>>
>>>> ```
>>>>
>>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>>
>>>> final
>>>>
>>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>>> aggValueSerde)
>>>>
>>>> ```
>>>>
>>>>
>>>> If we do not have the initializer in the first co-group it might be a bit
>>>> awkward for users to specify the aggregator that returns a typed <T>
>>>>
>>>> value?
>>>>
>>>> Maybe it is still better to put these two functions in the same api?
>>>>
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>> This suggestion lgtm. I would vote for the first alternative than
>>>>
>>>> adding
>>>>
>>>> it to the `KStreamBuilder` though.
>>>>
>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io>
>>>> wrote:
>>>>
>>>>
>>>> I have a minor suggestion to make the API a little bit more symmetric.
>>>> I feel it would make more sense to move the initializer and serde to
>>>>
>>>> the
>>>>
>>>> final aggregate statement, since the serde only applies to the state
>>>> store,
>>>> and the initializer doesn't bear any relation to the first group in
>>>> particular. It would end up looking like this:
>>>>
>>>> KTable<K, CG> cogrouped =
>>>>      grouped1.cogroup(aggregator1)
>>>>              .cogroup(grouped2, aggregator2)
>>>>              .cogroup(grouped3, aggregator3)
>>>>              .aggregate(initializer1, aggValueSerde, storeName1);
>>>>
>>>> Alternatively, we could move the the first cogroup() method to
>>>> KStreamBuilder, similar to how we have .merge()
>>>> and end up with an api that would be even more symmetric.
>>>>
>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>                .cogroup(grouped2, aggregator2)
>>>>                .cogroup(grouped3, aggregator3)
>>>>                .aggregate(initializer1, aggValueSerde, storeName1);
>>>>
>>>> This doesn't have to be a blocker, but I thought it would make the API
>>>> just
>>>> a tad cleaner.
>>>>
>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>> Kyle,
>>>>
>>>> Thanks a lot for the updated KIP. It looks good to me.
>>>>
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <ji...@jagunet.com>
>>>>
>>>> wrote:
>>>>
>>>> This makes much more sense to me. +1
>>>>
>>>>
>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>
>>>> winkelman.kyle@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>> I have updated the KIP and my PR. Let me know what you think.
>>>> To created a cogrouped stream just call cogroup on a
>>>>
>>>> KgroupedStream
>>>>
>>>> and
>>>>
>>>> supply the initializer, aggValueSerde, and an aggregator. Then
>>>>
>>>> continue
>>>>
>>>> adding kgroupedstreams and aggregators. Then call one of the
>>>>
>>>> many
>>>>
>>>> aggregate
>>>>
>>>> calls to create a KTable.
>>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <da...@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>> Hi Kyle,
>>>>
>>>> Thanks for the update. I think just one initializer makes sense
>>>>
>>>> as
>>>>
>>>> it
>>>>
>>>> should only be called once per key and generally it is just
>>>>
>>>> going
>>>>
>>>> to
>>>>
>>>> create
>>>>
>>>> a new instance of whatever the Aggregate class is.
>>>>
>>>> Cheers,
>>>> Damian
>>>>
>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>
>>>> winkelman.kyle@gmail.com
>>>>
>>>> wrote:
>>>>
>>>>
>>>> Hello all,
>>>>
>>>> I have spent some more time on this and the best alternative I
>>>>
>>>> have
>>>>
>>>> come
>>>>
>>>> up
>>>>
>>>> with is:
>>>> KGroupedStream has a single cogroup call that takes an
>>>>
>>>> initializer
>>>>
>>>> and
>>>>
>>>> an
>>>>
>>>> aggregator.
>>>> CogroupedKStream has a cogroup call that takes additional
>>>>
>>>> groupedStream
>>>>
>>>> aggregator pairs.
>>>> CogroupedKStream has multiple aggregate methods that create
>>>>
>>>> the
>>>>
>>>> different
>>>>
>>>> stores.
>>>>
>>>> I plan on updating the kip but I want people's input on if we
>>>>
>>>> should
>>>>
>>>> have
>>>>
>>>> the initializer be passed in once at the beginning or if we
>>>>
>>>> should
>>>>
>>>> instead
>>>>
>>>> have the initializer be required for each call to one of the
>>>>
>>>> aggregate
>>>>
>>>> calls. The first makes more sense to me but doesnt allow the
>>>>
>>>> user
>>>>
>>>> to
>>>>
>>>> specify different initializers for different tables.
>>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>
>>>> winkelman.kyle@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>>
>>>> Yea I really like that idea I'll see what I can do to update
>>>>
>>>> the
>>>>
>>>> kip
>>>>
>>>> and
>>>>
>>>> my pr when I have some time. I'm not sure how well creating
>>>>
>>>> the
>>>>
>>>> kstreamaggregates will go though because at that point I will
>>>>
>>>> have
>>>>
>>>> thrown
>>>>
>>>> away the type of the values. It will be type safe I just may
>>>>
>>>> need to
>>>>
>>>> do a
>>>>
>>>> little forcing.
>>>>
>>>> Thanks,
>>>> Kyle
>>>>
>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>>>>
>>>> wrote:
>>>>
>>>> Kyle,
>>>>
>>>> Thanks for the explanations, my previous read on the wiki
>>>>
>>>> examples
>>>>
>>>> was
>>>>
>>>> wrong.
>>>>
>>>> So I guess my motivation should be "reduced" to: can we move
>>>>
>>>> the
>>>>
>>>> window
>>>>
>>>> specs param from "KGroupedStream#cogroup(..)" to
>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>
>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>>>>
>>>> from
>>>>
>>>> 3
>>>>
>>>> to
>>>>
>>>> 2.
>>>>
>>>> 2. major: this is for extensibility of the APIs, and since
>>>>
>>>> we
>>>>
>>>> are
>>>>
>>>> removing
>>>>
>>>> the "Evolving" annotations on Streams it may be harder to
>>>>
>>>> change it
>>>>
>>>> again
>>>>
>>>> in the future. The extended use cases are that people wanted
>>>>
>>>> to
>>>>
>>>> have
>>>>
>>>> windowed running aggregates on different granularities, e.g.
>>>>
>>>> "give
>>>>
>>>> me
>>>>
>>>> the
>>>>
>>>> counts per-minute, per-hour, per-day and per-week", and
>>>>
>>>> today
>>>>
>>>> in
>>>>
>>>> DSL
>>>>
>>>> we
>>>>
>>>> need to specify that case in multiple aggregate operators,
>>>>
>>>> which
>>>>
>>>> gets
>>>>
>>>> a
>>>>
>>>> state store / changelog, etc. And it is possible to optimize
>>>>
>>>> it
>>>>
>>>> as
>>>>
>>>> well
>>>>
>>>> to
>>>>
>>>> a single state store. Its implementation would be tricky as
>>>>
>>>> you
>>>>
>>>> need
>>>>
>>>> to
>>>>
>>>> contain different lengthed windows within your window store
>>>>
>>>> but
>>>>
>>>> just
>>>>
>>>> from
>>>>
>>>> the public API point of view, it could be specified as:
>>>>
>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>> "state-store-name");
>>>>
>>>> table1 = stream.aggregate(/*per-minute window*/)
>>>> table2 = stream.aggregate(/*per-hour window*/)
>>>> table3 = stream.aggregate(/*per-day window*/)
>>>>
>>>> while underlying we are only using a single store
>>>>
>>>> "state-store-name"
>>>>
>>>> for
>>>>
>>>> it.
>>>>
>>>>
>>>> Although this feature is out of the scope of this KIP, I'd
>>>>
>>>> like
>>>>
>>>> to
>>>>
>>>> discuss
>>>>
>>>> if we can "leave the door open" to make such changes without
>>>>
>>>> modifying
>>>>
>>>> the
>>>>
>>>> public APIs .
>>>>
>>>> Guozhang
>>>>
>>>>
>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>
>>>> winkelman.kyle@gmail.com
>>>>
>>>> wrote:
>>>>
>>>>
>>>> I allow defining a single window/sessionwindow one time
>>>>
>>>> when
>>>>
>>>> you
>>>>
>>>> make
>>>>
>>>> the
>>>>
>>>> cogroup call from a KGroupedStream. From then on you are
>>>>
>>>> using
>>>>
>>>> the
>>>>
>>>> cogroup
>>>>
>>>> call from with in CogroupedKStream which doesnt accept any
>>>>
>>>> additional
>>>>
>>>> windows/sessionwindows.
>>>>
>>>> Is this what you meant by your question or did I
>>>>
>>>> misunderstand?
>>>>
>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>>
>>>> wangguoz@gmail.com
>>>>
>>>> wrote:
>>>>
>>>> Another question that came to me is on "window alignment":
>>>>
>>>> from
>>>>
>>>> the
>>>>
>>>> KIP
>>>>
>>>> it
>>>>
>>>> seems you are allowing users to specify a (potentially
>>>>
>>>> different)
>>>>
>>>> window
>>>>
>>>> spec in each co-grouped input stream. So if these window
>>>>
>>>> specs
>>>>
>>>> are
>>>>
>>>> different how should we "align" them with different input
>>>>
>>>> streams? I
>>>>
>>>> think
>>>>
>>>> it is more natural to only specify on window spec in the
>>>>
>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>
>>>>
>>>> And remove it from the cogroup() functions. WDYT?
>>>>
>>>>
>>>> Guozhang
>>>>
>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>
>>>> wangguoz@gmail.com>
>>>>
>>>> wrote:
>>>>
>>>> Thanks for the proposal Kyle, this is a quite common use
>>>>
>>>> case
>>>>
>>>> to
>>>>
>>>> support
>>>>
>>>> such multi-way table join (i.e. N source tables with N
>>>>
>>>> aggregate
>>>>
>>>> func)
>>>>
>>>> with
>>>>
>>>> a single store and N+1 serdes, I have seen lots of people
>>>>
>>>> using
>>>>
>>>> the
>>>>
>>>> low-level PAPI to achieve this goal.
>>>>
>>>>
>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>
>>>> winkelman.kyle@gmail.com
>>>>
>>>> wrote:
>>>>
>>>> I like your point about not handling other cases such as
>>>>
>>>> count
>>>>
>>>> and
>>>>
>>>> reduce.
>>>>
>>>> I think that reduce may not make sense because reduce
>>>>
>>>> assumes
>>>>
>>>> that
>>>>
>>>> the
>>>>
>>>> input values are the same as the output values. With
>>>>
>>>> cogroup
>>>>
>>>> ...

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
First, I would prefer not calling it aggregate because there are already
plenty of aggregate methods.

Second, I dont think this would really work because after each aggregate
you now have a unique KTable (someone may want a table with 4 streams and
reuse those 4 in another table but with one more stream added) and unless
we completely duplicate everything every time this isnt really possible.
Additionally, the cogroup way just requires 1 more call to create two
different tables (normal, windowed, and session windowed) this new way
would require copying the aggregate chain.

Another way to think about it is with cogroup we know that when they call
aggregate they arent going to be adding any more aggregators to that table
but your way requires us to assume they are done adding aggregators after
each call so we must return a ktable just to possibly not need to have
created it.

On Jun 13, 2017 5:20 AM, "Michal Borowiecki" <mi...@openbet.com>
wrote:

> Actually, just had a thought. It started with naming.
>
> Are we actually co-grouping these streams or are we co-aggregating them?
>
> After all, in each of the cogroup calls we are providing an Aggregator
> implementation.
>
>
> If they are really co-aggregated, why don't we turn this around:
> KGroupedStream<K, V1> grouped1 = builder.stream("topic1").groupByKey();
> KGroupedStream<K, V2> grouped2 = builder.stream("topic2").groupByKey();
> KGroupedStream<K, V3> grouped3 = builder.stream("topic3").groupByKey();
>
> KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1,
> aggValueSerde1) // this is the unchanged aggregate method
>         .aggregate(grouped2, aggregator2)  // this is a new method
>         .aggregate(grouped3, aggregator3); // ditto
>
> This means instead of adding cogroup methods on KGroupStream interface,
> adding aggregate method on KTable interface.
>
> Is that feasible?
>
> Cheers,
> Michał
>
> On 13/06/17 10:56, Michal Borowiecki wrote:
>
> Also, I still feel that putting initializer on the first cogroup can
> mislead users into thinking the first stream is in some way special.
> Just my 5c.
> Michał
>
> On 13/06/17 09:54, Michal Borowiecki wrote:
>
> Agree completely with the argument for serdes belonging in the same place
> as the state store name, which is in the aggregate method.
>
> Cheers,
>
> Michał
>
> On 12/06/17 18:20, Xavier Léauté wrote:
>
> I think we are discussing two separate things here, so it might be worth
> clarifying:
>
> 1) the position of the initializer with respect to the aggregators. If I
> understand correctly, Guozhang seems to think it is more natural to specify
> the initializer first, despite it not bearing any relation to the first
> aggregator. I can see the argument for specifying the initializer first,
> but I think it is debatable whether mixing it into the first cogroup call
> leads to a cleaner API or not.
>
> 2) where the serde should be defined (if necessary). Looking at our
> existing APIs in KGroupedStreams, we always offer two aggregate()
> methods. The first one takes the name of the store and associated aggregate
> value serde e.g. KGroupedStream.aggregate(Initializer<VR> initializer,
> Aggregator<? super K, ? super V, VR> aggregator, Serde<VR> aggValueSerde,
> String queryableStoreName)
> The second one only takes a state store supplier, and does not specify any
> serde, e.g. KGroupedStream.aggregate(Initializer<VR>
> initializer, Aggregator<? super K, ? super V, VR> aggregator, final
> StateStoreSupplier<KeyValueStore> storeSupplier)
> Presumably, when specifying a state store supplier it shouldn't be
> necessary to specify an aggregate value serde, since the provided
> statestore might not need to serialize the values (e.g. it may just keep
> them as regular objects in heap) or it may have its own
> internal serialization format.
>
> For consistency I think it would be valuable to preserve the same two
> aggregate methods for cogroup as well. Since the serde is only required in
> one of the two cases, I believe the serde has no place in the first
> cogroup() call and should only have to be specified as part of the
> aggregate() method that takes a state store name. In the case of a state
> store supplier, no serde would be necessary.
>
>
> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com> wrote:
>
>> I'd agree that the aggregate value serde and the initializer does not
>> bear direct relationship with the first `cogroup` calls, but after I tried
>> to write some example code with these two different set of APIs I felt the
>> current APIs just program more naturally.
>>
>> I know it is kinda subjective, but I do think that user experience may be
>> more important as a deciding factor than the logical argument for public
>> interfaces. So I'd recommend people to also try out writing some example
>> lines also and we can circle back and discuss which one feels more natural
>> to write code.
>>
>>
>> Guozhang
>>
>> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
>> michal.borowiecki@openbet.com> wrote:
>>
>>> I feel it would make more sense to move the initializer and serde to the
>>> final aggregate statement, since the serde only applies to the state
>>> store,
>>> and the initializer doesn't bear any relation to the first group in
>>> particular.
>>>
>>> +1 for moving initializer and serde from cogroup() to the aggregate()
>>> for the reasons mentioned above.
>>>
>>> Cheers,
>>>
>>> Michał
>>>
>>> On 08/06/17 22:44, Guozhang Wang wrote:
>>>
>> Note that although the internal `AbstractStoreSupplier` does maintain the
>>> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>> to always retain that information, and hence we cannot assume that
>>> StateStoreSuppliers always retain key / value serdes.
>>>
>>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io> wrote:
>>>
>>>
>>> Another reason for the serde not to be in the first cogroup call, is that
>>> the serde should not be required if you pass a StateStoreSupplier to
>>> aggregate()
>>>
>>> Regarding the aggregated type <T> I don't the why initializer should be
>>> favored over aggregator to define the type. In my mind separating the
>>> initializer into the last aggregate call clearly indicates that the
>>> initializer is independent of any of the aggregators or streams and that we
>>> don't wait for grouped1 events to initialize the co-group.
>>>
>>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com> wrote:
>>>
>>>
>>> On a second thought... This is the current proposal API
>>>
>>>
>>> ```
>>>
>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>
>>> final
>>>
>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>> aggValueSerde)
>>>
>>> ```
>>>
>>>
>>> If we do not have the initializer in the first co-group it might be a bit
>>> awkward for users to specify the aggregator that returns a typed <T>
>>>
>>> value?
>>>
>>> Maybe it is still better to put these two functions in the same api?
>>>
>>>
>>>
>>> Guozhang
>>>
>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>
>>> wrote:
>>>
>>> This suggestion lgtm. I would vote for the first alternative than
>>>
>>> adding
>>>
>>> it to the `KStreamBuilder` though.
>>>
>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io>
>>> wrote:
>>>
>>>
>>> I have a minor suggestion to make the API a little bit more symmetric.
>>> I feel it would make more sense to move the initializer and serde to
>>>
>>> the
>>>
>>> final aggregate statement, since the serde only applies to the state
>>> store,
>>> and the initializer doesn't bear any relation to the first group in
>>> particular. It would end up looking like this:
>>>
>>> KTable<K, CG> cogrouped =
>>>     grouped1.cogroup(aggregator1)
>>>             .cogroup(grouped2, aggregator2)
>>>             .cogroup(grouped3, aggregator3)
>>>             .aggregate(initializer1, aggValueSerde, storeName1);
>>>
>>> Alternatively, we could move the the first cogroup() method to
>>> KStreamBuilder, similar to how we have .merge()
>>> and end up with an api that would be even more symmetric.
>>>
>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>>               .cogroup(grouped2, aggregator2)
>>>               .cogroup(grouped3, aggregator3)
>>>               .aggregate(initializer1, aggValueSerde, storeName1);
>>>
>>> This doesn't have to be a blocker, but I thought it would make the API
>>> just
>>> a tad cleaner.
>>>
>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>>
>>> wrote:
>>>
>>> Kyle,
>>>
>>> Thanks a lot for the updated KIP. It looks good to me.
>>>
>>>
>>> Guozhang
>>>
>>>
>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <ji...@jagunet.com>
>>>
>>> wrote:
>>>
>>> This makes much more sense to me. +1
>>>
>>>
>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>
>>> winkelman.kyle@gmail.com>
>>>
>>> wrote:
>>>
>>> I have updated the KIP and my PR. Let me know what you think.
>>> To created a cogrouped stream just call cogroup on a
>>>
>>> KgroupedStream
>>>
>>> and
>>>
>>> supply the initializer, aggValueSerde, and an aggregator. Then
>>>
>>> continue
>>>
>>> adding kgroupedstreams and aggregators. Then call one of the
>>>
>>> many
>>>
>>> aggregate
>>>
>>> calls to create a KTable.
>>>
>>> Thanks,
>>> Kyle
>>>
>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <da...@gmail.com>
>>>
>>> wrote:
>>>
>>> Hi Kyle,
>>>
>>> Thanks for the update. I think just one initializer makes sense
>>>
>>> as
>>>
>>> it
>>>
>>> should only be called once per key and generally it is just
>>>
>>> going
>>>
>>> to
>>>
>>> create
>>>
>>> a new instance of whatever the Aggregate class is.
>>>
>>> Cheers,
>>> Damian
>>>
>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>
>>> winkelman.kyle@gmail.com
>>>
>>> wrote:
>>>
>>>
>>> Hello all,
>>>
>>> I have spent some more time on this and the best alternative I
>>>
>>> have
>>>
>>> come
>>>
>>> up
>>>
>>> with is:
>>> KGroupedStream has a single cogroup call that takes an
>>>
>>> initializer
>>>
>>> and
>>>
>>> an
>>>
>>> aggregator.
>>> CogroupedKStream has a cogroup call that takes additional
>>>
>>> groupedStream
>>>
>>> aggregator pairs.
>>> CogroupedKStream has multiple aggregate methods that create
>>>
>>> the
>>>
>>> different
>>>
>>> stores.
>>>
>>> I plan on updating the kip but I want people's input on if we
>>>
>>> should
>>>
>>> have
>>>
>>> the initializer be passed in once at the beginning or if we
>>>
>>> should
>>>
>>> instead
>>>
>>> have the initializer be required for each call to one of the
>>>
>>> aggregate
>>>
>>> calls. The first makes more sense to me but doesnt allow the
>>>
>>> user
>>>
>>> to
>>>
>>> specify different initializers for different tables.
>>>
>>> Thanks,
>>> Kyle
>>>
>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>
>>> winkelman.kyle@gmail.com>
>>>
>>> wrote:
>>>
>>>
>>> Yea I really like that idea I'll see what I can do to update
>>>
>>> the
>>>
>>> kip
>>>
>>> and
>>>
>>> my pr when I have some time. I'm not sure how well creating
>>>
>>> the
>>>
>>> kstreamaggregates will go though because at that point I will
>>>
>>> have
>>>
>>> thrown
>>>
>>> away the type of the values. It will be type safe I just may
>>>
>>> need to
>>>
>>> do a
>>>
>>> little forcing.
>>>
>>> Thanks,
>>> Kyle
>>>
>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>>>
>>> wrote:
>>>
>>> Kyle,
>>>
>>> Thanks for the explanations, my previous read on the wiki
>>>
>>> examples
>>>
>>> was
>>>
>>> wrong.
>>>
>>> So I guess my motivation should be "reduced" to: can we move
>>>
>>> the
>>>
>>> window
>>>
>>> specs param from "KGroupedStream#cogroup(..)" to
>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>>
>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>>>
>>> from
>>>
>>> 3
>>>
>>> to
>>>
>>> 2.
>>>
>>> 2. major: this is for extensibility of the APIs, and since
>>>
>>> we
>>>
>>> are
>>>
>>> removing
>>>
>>> the "Evolving" annotations on Streams it may be harder to
>>>
>>> change it
>>>
>>> again
>>>
>>> in the future. The extended use cases are that people wanted
>>>
>>> to
>>>
>>> have
>>>
>>> windowed running aggregates on different granularities, e.g.
>>>
>>> "give
>>>
>>> me
>>>
>>> the
>>>
>>> counts per-minute, per-hour, per-day and per-week", and
>>>
>>> today
>>>
>>> in
>>>
>>> DSL
>>>
>>> we
>>>
>>> need to specify that case in multiple aggregate operators,
>>>
>>> which
>>>
>>> gets
>>>
>>> a
>>>
>>> state store / changelog, etc. And it is possible to optimize
>>>
>>> it
>>>
>>> as
>>>
>>> well
>>>
>>> to
>>>
>>> a single state store. Its implementation would be tricky as
>>>
>>> you
>>>
>>> need
>>>
>>> to
>>>
>>> contain different lengthed windows within your window store
>>>
>>> but
>>>
>>> just
>>>
>>> from
>>>
>>> the public API point of view, it could be specified as:
>>>
>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>>> "state-store-name");
>>>
>>> table1 = stream.aggregate(/*per-minute window*/)
>>> table2 = stream.aggregate(/*per-hour window*/)
>>> table3 = stream.aggregate(/*per-day window*/)
>>>
>>> while underlying we are only using a single store
>>>
>>> "state-store-name"
>>>
>>> for
>>>
>>> it.
>>>
>>>
>>> Although this feature is out of the scope of this KIP, I'd
>>>
>>> like
>>>
>>> to
>>>
>>> discuss
>>>
>>> if we can "leave the door open" to make such changes without
>>>
>>> modifying
>>>
>>> the
>>>
>>> public APIs .
>>>
>>> Guozhang
>>>
>>>
>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>
>>> winkelman.kyle@gmail.com
>>>
>>> wrote:
>>>
>>>
>>> I allow defining a single window/sessionwindow one time
>>>
>>> when
>>>
>>> you
>>>
>>> make
>>>
>>> the
>>>
>>> cogroup call from a KGroupedStream. From then on you are
>>>
>>> using
>>>
>>> the
>>>
>>> cogroup
>>>
>>> call from with in CogroupedKStream which doesnt accept any
>>>
>>> additional
>>>
>>> windows/sessionwindows.
>>>
>>> Is this what you meant by your question or did I
>>>
>>> misunderstand?
>>>
>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>
>>> wangguoz@gmail.com
>>>
>>> wrote:
>>>
>>> Another question that came to me is on "window alignment":
>>>
>>> from
>>>
>>> the
>>>
>>> KIP
>>>
>>> it
>>>
>>> seems you are allowing users to specify a (potentially
>>>
>>> different)
>>>
>>> window
>>>
>>> spec in each co-grouped input stream. So if these window
>>>
>>> specs
>>>
>>> are
>>>
>>> different how should we "align" them with different input
>>>
>>> streams? I
>>>
>>> think
>>>
>>> it is more natural to only specify on window spec in the
>>>
>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>
>>>
>>> And remove it from the cogroup() functions. WDYT?
>>>
>>>
>>> Guozhang
>>>
>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>
>>> wangguoz@gmail.com>
>>>
>>> wrote:
>>>
>>> Thanks for the proposal Kyle, this is a quite common use
>>>
>>> case
>>>
>>> to
>>>
>>> support
>>>
>>> such multi-way table join (i.e. N source tables with N
>>>
>>> aggregate
>>>
>>> func)
>>>
>>> with
>>>
>>> a single store and N+1 serdes, I have seen lots of people
>>>
>>> using
>>>
>>> the
>>>
>>> low-level PAPI to achieve this goal.
>>>
>>>
>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>
>>> winkelman.kyle@gmail.com
>>>
>>> wrote:
>>>
>>> I like your point about not handling other cases such as
>>>
>>> count
>>>
>>> and
>>>
>>> reduce.
>>>
>>> I think that reduce may not make sense because reduce
>>>
>>> assumes
>>>
>>> that
>>>
>>> the
>>>
>>> input values are the same as the output values. With
>>>
>>> cogroup
>>>
>>> ...

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Michal Borowiecki <mi...@openbet.com>.
Actually, just had a thought. It started with naming.

Are we actually co-grouping these streams or are we co-aggregating them?

After all, in each of the cogroup calls we are providing an Aggregator 
implementation.


If they are really co-aggregated, why don't we turn this around:

|KGroupedStream<K, V1> grouped1 = 
builder.stream(||"topic1"||).groupByKey();|
|KGroupedStream<K, V2> grouped2 = 
builder.stream(||"topic2"||).groupByKey();|
|KGroupedStream<K, V3> grouped3 = builder.stream(||"topic3"||).groupByKey();
|
|KTable<K, CG> coagg = grouped1.aggregate(initializer1, aggregator1, 
aggValueSerde1) // this is the unchanged aggregate method||
||.aggregate(grouped2, aggregator2)  // this is a new method
|
|||.aggregate(grouped3, aggregator3); // ditto
|
|
|
This means instead of adding cogroup methods on KGroupStream interface, 
adding aggregate method on KTable interface.

Is that feasible?

Cheers,
Michał

On 13/06/17 10:56, Michal Borowiecki wrote:
>
> Also, I still feel that putting initializer on the first cogroup can 
> mislead users into thinking the first stream is in some way special.
>
> Just my 5c.
> Michał
>
> On 13/06/17 09:54, Michal Borowiecki wrote:
>>
>> Agree completely with the argument for serdes belonging in the same 
>> place as the state store name, which is in the aggregate method.
>>
>> Cheers,
>>
>> Michał
>>
>>
>> On 12/06/17 18:20, Xavier Léauté wrote:
>>> I think we are discussing two separate things here, so it might be 
>>> worth clarifying:
>>>
>>> 1) the position of the initializer with respect to the aggregators. 
>>> If I understand correctly, Guozhang seems to think it is more 
>>> natural to specify the initializer first, despite it not bearing any 
>>> relation to the first aggregator. I can see the argument for 
>>> specifying the initializer first, but I think it is debatable 
>>> whether mixing it into the first cogroup call leads to a cleaner API 
>>> or not.
>>>
>>> 2) where the serde should be defined (if necessary). Looking at our 
>>> existing APIs in KGroupedStreams, we always offer two aggregate() 
>>> methods. The first one takes the name of the store and associated 
>>> aggregate value serde e.g. KGroupedStream.aggregate(Initializer<VR> 
>>> initializer, Aggregator<? super K, ? super V, VR> aggregator, 
>>> Serde<VR> aggValueSerde, String queryableStoreName)
>>> The second one only takes a state store supplier, and does not 
>>> specify any serde, e.g. KGroupedStream.aggregate(Initializer<VR> 
>>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final 
>>> StateStoreSupplier<KeyValueStore> storeSupplier)
>>> Presumably, when specifying a state store supplier it shouldn't be 
>>> necessary to specify an aggregate value serde, since the provided 
>>> statestore might not need to serialize the values (e.g. it may just 
>>> keep them as regular objects in heap) or it may have its own 
>>> internal serialization format.
>>>
>>> For consistency I think it would be valuable to preserve the same 
>>> two aggregate methods for cogroup as well. Since the serde is only 
>>> required in one of the two cases, I believe the serde has no place 
>>> in the first cogroup() call and should only have to be specified as 
>>> part of the aggregate() method that takes a state store name. In the 
>>> case of a state store supplier, no serde would be necessary.
>>>
>>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wangguoz@gmail.com 
>>> <ma...@gmail.com>> wrote:
>>>
>>>     I'd agree that the aggregate value serde and the initializer
>>>     does not bear direct relationship with the first `cogroup`
>>>     calls, but after I tried to write some example code with these
>>>     two different set of APIs I felt the current APIs just program
>>>     more naturally.
>>>
>>>     I know it is kinda subjective, but I do think that user
>>>     experience may be more important as a deciding factor than the
>>>     logical argument for public interfaces. So I'd recommend people
>>>     to also try out writing some example lines also and we can
>>>     circle back and discuss which one feels more natural to write code.
>>>
>>>
>>>     Guozhang
>>>
>>>     On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki
>>>     <michal.borowiecki@openbet.com
>>>     <ma...@openbet.com>> wrote:
>>>
>>>>         I feel it would make more sense to move the initializer and
>>>>         serde to the
>>>>         final aggregate statement, since the serde only applies to
>>>>         the state store,
>>>>         and the initializer doesn't bear any relation to the first
>>>>         group in
>>>>         particular.
>>>         +1 for moving initializer and serde from cogroup() to the
>>>         aggregate() for the reasons mentioned above.
>>>
>>>         Cheers,
>>>
>>>         Michał
>>>
>>>
>>>         On 08/06/17 22:44, Guozhang Wang wrote:
>>>
>>>>         Note that although the internal `AbstractStoreSupplier` does maintain the
>>>>         key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>>>         to always retain that information, and hence we cannot assume that
>>>>         StateStoreSuppliers always retain key / value serdes.
>>>>
>>>>         On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>  wrote:
>>>>
>>>>>         Another reason for the serde not to be in the first cogroup call, is that
>>>>>         the serde should not be required if you pass a StateStoreSupplier to
>>>>>         aggregate()
>>>>>
>>>>>         Regarding the aggregated type <T> I don't the why initializer should be
>>>>>         favored over aggregator to define the type. In my mind separating the
>>>>>         initializer into the last aggregate call clearly indicates that the
>>>>>         initializer is independent of any of the aggregators or streams and that we
>>>>>         don't wait for grouped1 events to initialize the co-group.
>>>>>
>>>>>         On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>  wrote:
>>>>>
>>>>>>         On a second thought... This is the current proposal API
>>>>>>
>>>>>>
>>>>>>         ```
>>>>>>
>>>>>>         <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>>>         final
>>>>>>         Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>>>>>         aggValueSerde)
>>>>>>
>>>>>>         ```
>>>>>>
>>>>>>
>>>>>>         If we do not have the initializer in the first co-group it might be a bit
>>>>>>         awkward for users to specify the aggregator that returns a typed <T>
>>>>>         value?
>>>>>>         Maybe it is still better to put these two functions in the same api?
>>>>>>
>>>>>>
>>>>>>
>>>>>>         Guozhang
>>>>>>
>>>>>>         On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>>>         wrote:
>>>>>>>         This suggestion lgtm. I would vote for the first alternative than
>>>>>         adding
>>>>>>>         it to the `KStreamBuilder` though.
>>>>>>>
>>>>>>>         On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>
>>>>>>>         wrote:
>>>>>>>
>>>>>>>>         I have a minor suggestion to make the API a little bit more symmetric.
>>>>>>>>         I feel it would make more sense to move the initializer and serde to
>>>>>         the
>>>>>>>>         final aggregate statement, since the serde only applies to the state
>>>>>>>>         store,
>>>>>>>>         and the initializer doesn't bear any relation to the first group in
>>>>>>>>         particular. It would end up looking like this:
>>>>>>>>
>>>>>>>>         KTable<K, CG> cogrouped =
>>>>>>>>              grouped1.cogroup(aggregator1)
>>>>>>>>                      .cogroup(grouped2, aggregator2)
>>>>>>>>                      .cogroup(grouped3, aggregator3)
>>>>>>>>                      .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>>>
>>>>>>>>         Alternatively, we could move the the first cogroup() method to
>>>>>>>>         KStreamBuilder, similar to how we have .merge()
>>>>>>>>         and end up with an api that would be even more symmetric.
>>>>>>>>
>>>>>>>>         KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>>>>>                        .cogroup(grouped2, aggregator2)
>>>>>>>>                        .cogroup(grouped3, aggregator3)
>>>>>>>>                        .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>>>
>>>>>>>>         This doesn't have to be a blocker, but I thought it would make the API
>>>>>>>>         just
>>>>>>>>         a tad cleaner.
>>>>>>>>
>>>>>>>>         On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>>>>         wrote:
>>>>>>>>>         Kyle,
>>>>>>>>>
>>>>>>>>>         Thanks a lot for the updated KIP. It looks good to me.
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>         Guozhang
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>         On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski<ji...@jagunet.com> <ma...@jagunet.com>
>>>>>>         wrote:
>>>>>>>>>>         This makes much more sense to me. +1
>>>>>>>>>>
>>>>>>>>>>>         On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>>>         wrote:
>>>>>>>>>>>         I have updated the KIP and my PR. Let me know what you think.
>>>>>>>>>>>         To created a cogrouped stream just call cogroup on a
>>>>>>         KgroupedStream
>>>>>>>>         and
>>>>>>>>>>>         supply the initializer, aggValueSerde, and an aggregator. Then
>>>>>>>>         continue
>>>>>>>>>>>         adding kgroupedstreams and aggregators. Then call one of the
>>>>>         many
>>>>>>>>>>         aggregate
>>>>>>>>>>>         calls to create a KTable.
>>>>>>>>>>>
>>>>>>>>>>>         Thanks,
>>>>>>>>>>>         Kyle
>>>>>>>>>>>
>>>>>>>>>>>         On Jun 1, 2017 4:03 AM, "Damian Guy"<da...@gmail.com> <ma...@gmail.com>
>>>>>>         wrote:
>>>>>>>>>>>>         Hi Kyle,
>>>>>>>>>>>>
>>>>>>>>>>>>         Thanks for the update. I think just one initializer makes sense
>>>>>>         as
>>>>>>>>         it
>>>>>>>>>>>>         should only be called once per key and generally it is just
>>>>>         going
>>>>>>>>         to
>>>>>>>>>>         create
>>>>>>>>>>>>         a new instance of whatever the Aggregate class is.
>>>>>>>>>>>>
>>>>>>>>>>>>         Cheers,
>>>>>>>>>>>>         Damian
>>>>>>>>>>>>
>>>>>>>>>>>>         On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>         Hello all,
>>>>>>>>>>>>>
>>>>>>>>>>>>>         I have spent some more time on this and the best alternative I
>>>>>>>>         have
>>>>>>>>>>         come
>>>>>>>>>>>>         up
>>>>>>>>>>>>>         with is:
>>>>>>>>>>>>>         KGroupedStream has a single cogroup call that takes an
>>>>>>         initializer
>>>>>>>>>         and
>>>>>>>>>>         an
>>>>>>>>>>>>>         aggregator.
>>>>>>>>>>>>>         CogroupedKStream has a cogroup call that takes additional
>>>>>>>>>         groupedStream
>>>>>>>>>>>>>         aggregator pairs.
>>>>>>>>>>>>>         CogroupedKStream has multiple aggregate methods that create
>>>>>         the
>>>>>>>>>>         different
>>>>>>>>>>>>>         stores.
>>>>>>>>>>>>>
>>>>>>>>>>>>>         I plan on updating the kip but I want people's input on if we
>>>>>>>>         should
>>>>>>>>>>         have
>>>>>>>>>>>>>         the initializer be passed in once at the beginning or if we
>>>>>>         should
>>>>>>>>>>>>         instead
>>>>>>>>>>>>>         have the initializer be required for each call to one of the
>>>>>>>>>         aggregate
>>>>>>>>>>>>>         calls. The first makes more sense to me but doesnt allow the
>>>>>>         user
>>>>>>>>         to
>>>>>>>>>>>>>         specify different initializers for different tables.
>>>>>>>>>>>>>
>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>
>>>>>>>>>>>>>         On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Yea I really like that idea I'll see what I can do to update
>>>>>>         the
>>>>>>>>         kip
>>>>>>>>>>>>         and
>>>>>>>>>>>>>>         my pr when I have some time. I'm not sure how well creating
>>>>>         the
>>>>>>>>>>>>>>         kstreamaggregates will go though because at that point I will
>>>>>>>>         have
>>>>>>>>>>>>         thrown
>>>>>>>>>>>>>>         away the type of the values. It will be type safe I just may
>>>>>>>>         need to
>>>>>>>>>>>>         do a
>>>>>>>>>>>>>>         little forcing.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>         Kyle,
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         Thanks for the explanations, my previous read on the wiki
>>>>>>>>         examples
>>>>>>>>>>         was
>>>>>>>>>>>>>>>         wrong.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         So I guess my motivation should be "reduced" to: can we move
>>>>>>         the
>>>>>>>>>>>>         window
>>>>>>>>>>>>>>>         specs param from "KGroupedStream#cogroup(..)" to
>>>>>>>>>>>>>>>         "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         1. minor: we can reduce the #.generics in CogroupedKStream
>>>>>>         from
>>>>>>>>         3
>>>>>>>>>         to
>>>>>>>>>>>>         2.
>>>>>>>>>>>>>>>         2. major: this is for extensibility of the APIs, and since
>>>>>         we
>>>>>>>>         are
>>>>>>>>>>>>>         removing
>>>>>>>>>>>>>>>         the "Evolving" annotations on Streams it may be harder to
>>>>>>>>         change it
>>>>>>>>>>>>>         again
>>>>>>>>>>>>>>>         in the future. The extended use cases are that people wanted
>>>>>>         to
>>>>>>>>>         have
>>>>>>>>>>>>>>>         windowed running aggregates on different granularities, e.g.
>>>>>>>>         "give
>>>>>>>>>         me
>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>         counts per-minute, per-hour, per-day and per-week", and
>>>>>         today
>>>>>>         in
>>>>>>>>>         DSL
>>>>>>>>>>>>         we
>>>>>>>>>>>>>>>         need to specify that case in multiple aggregate operators,
>>>>>>         which
>>>>>>>>>         gets
>>>>>>>>>>>>         a
>>>>>>>>>>>>>>>         state store / changelog, etc. And it is possible to optimize
>>>>>>         it
>>>>>>>>         as
>>>>>>>>>>>>         well
>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>         a single state store. Its implementation would be tricky as
>>>>>>         you
>>>>>>>>>         need
>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>         contain different lengthed windows within your window store
>>>>>>         but
>>>>>>>>>         just
>>>>>>>>>>>>>         from
>>>>>>>>>>>>>>>         the public API point of view, it could be specified as:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>>>>>>>>>>>>         "state-store-name");
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         table1 = stream.aggregate(/*per-minute window*/)
>>>>>>>>>>>>>>>         table2 = stream.aggregate(/*per-hour window*/)
>>>>>>>>>>>>>>>         table3 = stream.aggregate(/*per-day window*/)
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         while underlying we are only using a single store
>>>>>>>>>         "state-store-name"
>>>>>>>>>>>>         for
>>>>>>>>>>>>>>>         it.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         Although this feature is out of the scope of this KIP, I'd
>>>>>>         like
>>>>>>>>         to
>>>>>>>>>>>>>         discuss
>>>>>>>>>>>>>>>         if we can "leave the door open" to make such changes without
>>>>>>>>>>         modifying
>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>         public APIs .
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         I allow defining a single window/sessionwindow one time
>>>>>         when
>>>>>>>>         you
>>>>>>>>>>>>         make
>>>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>         cogroup call from a KGroupedStream. From then on you are
>>>>>>         using
>>>>>>>>         the
>>>>>>>>>>>>>>>         cogroup
>>>>>>>>>>>>>>>>         call from with in CogroupedKStream which doesnt accept any
>>>>>>>>>>>>         additional
>>>>>>>>>>>>>>>>         windows/sessionwindows.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         Is this what you meant by your question or did I
>>>>>>         misunderstand?
>>>>>>>>>>>>>>>>         On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>>>         wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>         Another question that came to me is on "window alignment":
>>>>>>         from
>>>>>>>>>         the
>>>>>>>>>>>>>         KIP
>>>>>>>>>>>>>>>         it
>>>>>>>>>>>>>>>>         seems you are allowing users to specify a (potentially
>>>>>>>>         different)
>>>>>>>>>>>>>         window
>>>>>>>>>>>>>>>>         spec in each co-grouped input stream. So if these window
>>>>>>         specs
>>>>>>>>         are
>>>>>>>>>>>>>>>>         different how should we "align" them with different input
>>>>>>>>>         streams? I
>>>>>>>>>>>>>>>         think
>>>>>>>>>>>>>>>>         it is more natural to only specify on window spec in the
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         And remove it from the cogroup() functions. WDYT?
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>>>>>>         wangguoz@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>         Thanks for the proposal Kyle, this is a quite common use
>>>>>>         case
>>>>>>>>         to
>>>>>>>>>>>>>>>         support
>>>>>>>>>>>>>>>>>         such multi-way table join (i.e. N source tables with N
>>>>>>>>         aggregate
>>>>>>>>>>>>>         func)
>>>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>>>         a single store and N+1 serdes, I have seen lots of people
>>>>>>>>         using
>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>>         low-level PAPI to achieve this goal.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>>         I like your point about not handling other cases such as
>>>>>>>>         count
>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>         reduce.
>>>>>>>>>>>>>>>>>>         I think that reduce may not make sense because reduce
>>>>>>         assumes
>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>>>         input values are the same as the output values. With
>>>>>>         cogroup
>>>>>>>>>>>>         there
>>>>>>>>>>>>>>>         may
>>>>>>>>>>>>>>>>         be
>>>>>>>>>>>>>>>>>>         multiple different input types and then your output type
>>>>>>>>         cant be
>>>>>>>>>>>>>>>>         multiple
>>>>>>>>>>>>>>>>>>         different things. In the case where you have all matching
>>>>>>>>         value
>>>>>>>>>>>>>         types
>>>>>>>>>>>>>>>>         you
>>>>>>>>>>>>>>>>>>         can do KStreamBuilder#merge followed by the reduce.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         As for count I think it is possible to call count on all
>>>>>>         the
>>>>>>>>>>>>>>>         individual
>>>>>>>>>>>>>>>>>>         grouped streams and then do joins. Otherwise we could
>>>>>         maybe
>>>>>>>>         make
>>>>>>>>>>>>         a
>>>>>>>>>>>>>>>>         special
>>>>>>>>>>>>>>>>>>         call in groupedstream for this case. Because in this case
>>>>>>         we
>>>>>>>>>         dont
>>>>>>>>>>>>>>>         need
>>>>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>>>         do type checking on the values. It could be similar to
>>>>>         the
>>>>>>>>>>>>         current
>>>>>>>>>>>>>>>         count
>>>>>>>>>>>>>>>>>>         methods but accept a var args of additonal grouped
>>>>>         streams
>>>>>>         as
>>>>>>>>>>>>         well
>>>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>>>         make
>>>>>>>>>>>>>>>>>>         sure they have a key type of K.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         The way I have put the kip together is to ensure that we
>>>>>         do
>>>>>>>>         type
>>>>>>>>>>>>>>>>         checking.
>>>>>>>>>>>>>>>>>>         I don't see a way we could group them all first and then
>>>>>>>>         make a
>>>>>>>>>>>>>         call
>>>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>>>         count, reduce, or aggregate because with aggregate they
>>>>>>         would
>>>>>>>>>>>>         need
>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>         pass
>>>>>>>>>>>>>>>>>>         a list of aggregators and we would have no way of type
>>>>>>>>         checking
>>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>>         they
>>>>>>>>>>>>>>>>>>         match the grouped streams.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         On May 19, 2017 11:42 AM, "Xavier Léauté" <
>>>>>>>>         xavier@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>>>         Sorry to jump on this thread so late. I agree this is a
>>>>>>         very
>>>>>>>>>>>>>         useful
>>>>>>>>>>>>>>>>>>>         addition and wanted to provide an additional use-case
>>>>>         and
>>>>>>>>         some
>>>>>>>>>>>>>         more
>>>>>>>>>>>>>>>>>>>         comments.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         This is actually a very common analytics use-case in the
>>>>>>>>>>>>         ad-tech
>>>>>>>>>>>>>>>>>>         industry.
>>>>>>>>>>>>>>>>>>>         The typical setup will have an auction stream, an
>>>>>>         impression
>>>>>>>>>>>>>>>         stream,
>>>>>>>>>>>>>>>>>>         and a
>>>>>>>>>>>>>>>>>>>         click stream. Those three streams need to be combined to
>>>>>>>>>>>>         compute
>>>>>>>>>>>>>>>>>>         aggregate
>>>>>>>>>>>>>>>>>>>         statistics (e.g. impression statistics, and
>>>>>         click-through
>>>>>>>>>>>>         rates),
>>>>>>>>>>>>>>>>         since
>>>>>>>>>>>>>>>>>>>         most of the attributes of interest are only present the
>>>>>>>>         auction
>>>>>>>>>>>>>>>>         stream.
>>>>>>>>>>>>>>>>>>>         A simple way to do this is to co-group all the streams
>>>>>         by
>>>>>>>>         the
>>>>>>>>>>>>>>>         auction
>>>>>>>>>>>>>>>>>>         key,
>>>>>>>>>>>>>>>>>>>         and process updates to the co-group as events for each
>>>>>>>>         stream
>>>>>>>>>>>>>         come
>>>>>>>>>>>>>>>         in,
>>>>>>>>>>>>>>>>>>>         keeping only one value from each stream before sending
>>>>>>>>>>>>         downstream
>>>>>>>>>>>>>>>         for
>>>>>>>>>>>>>>>>>>>         further processing / aggregation.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         One could view the result of that co-group operation as
>>>>>         a
>>>>>>>>>>>>>         "KTable"
>>>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>>>>>         multiple values per key. The key being the grouping key,
>>>>>>         and
>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>         values
>>>>>>>>>>>>>>>>>>>         consisting of one value per stream.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         What I like about Kyle's approach is that allows elegant
>>>>>>>>>>>>>>>         co-grouping
>>>>>>>>>>>>>>>>         of
>>>>>>>>>>>>>>>>>>>         multiple streams without having to worry about the
>>>>>         number
>>>>>>         of
>>>>>>>>>>>>>>>         streams,
>>>>>>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>>>>         avoids dealing with Tuple types or other generic
>>>>>>         interfaces
>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>>>         get
>>>>>>>>>>>>>>>>>>>         messy if we wanted to preserve all the value types in
>>>>>         the
>>>>>>>>>>>>>         resulting
>>>>>>>>>>>>>>>>>>>         co-grouped stream.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         My only concern is that we only allow the cogroup +
>>>>>>>>         aggregate
>>>>>>>>>>>>>>>         combined
>>>>>>>>>>>>>>>>>>>         operation. This forces the user to build their own tuple
>>>>>>>>>>>>>>>         serialization
>>>>>>>>>>>>>>>>>>>         format if they want to preserve the individual input
>>>>>>         stream
>>>>>>>>>>>>>         values
>>>>>>>>>>>>>>>         as
>>>>>>>>>>>>>>>>         a
>>>>>>>>>>>>>>>>>>>         group. It also deviates quite a bit from our approach in
>>>>>>>>>>>>>>>>         KGroupedStream
>>>>>>>>>>>>>>>>>>>         which offers other operations, such as count and reduce,
>>>>>>>>         which
>>>>>>>>>>>>>>>         should
>>>>>>>>>>>>>>>>>>         also
>>>>>>>>>>>>>>>>>>>         be applicable to a co-grouped stream.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         Overall I still think this is a really useful addition,
>>>>>>         but
>>>>>>>>         I
>>>>>>>>>>>>>         feel
>>>>>>>>>>>>>>>         we
>>>>>>>>>>>>>>>>>>>         haven't spend much time trying to explore alternative
>>>>>         DSLs
>>>>>>>>         that
>>>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>>>         maybe
>>>>>>>>>>>>>>>>>>>         generalize better or match our existing syntax more
>>>>>>         closely.
>>>>>>>>>>>>>>>>>>>         On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         Eno, is there anyone else that is an expert in the
>>>
>>>>>         kafka
>>>>>>>>>>>>>         streams
>>>>>>>>>>>>>>>>         realm
>>>>>>>>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>>>>>>         I should reach out to for input?
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>         I believe Damian Guy is still planning on reviewing
>>>>>         this
>>>>>>>>         more
>>>>>>>>>>>>>         in
>>>>>>>>>>>>>>>>         depth
>>>>>>>>>>>>>>>>>>>         so I
>>>
>>
>> -- 
>> Signature
>> <http://www.openbet.com/> 	Michal Borowiecki
>> Senior Software Engineer L4
>> 	T: 	+44 208 742 1600
>>
>> 	
>> 	+44 203 249 8448
>>
>> 	
>> 	
>> 	E: 	michal.borowiecki@openbet.com
>> 	W: 	www.openbet.com <http://www.openbet.com/>
>>
>> 	
>> 	OpenBet Ltd
>>
>> 	Chiswick Park Building 9
>>
>> 	566 Chiswick High Rd
>>
>> 	London
>>
>> 	W4 5XT
>>
>> 	UK
>>
>> 	
>> <https://www.openbet.com/email_promo>
>>
>> This message is confidential and intended only for the addressee. If 
>> you have received this message in error, please immediately notify 
>> the postmaster@openbet.com <ma...@openbet.com> and delete 
>> it from your system as well as any copies. The content of e-mails as 
>> well as traffic data may be monitored by OpenBet for employment and 
>> security purposes. To protect the environment please do not print 
>> this e-mail unless necessary. OpenBet Ltd. Registered Office: 
>> Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT, 
>> United Kingdom. A company registered in England and Wales. Registered 
>> no. 3134634. VAT no. GB927523612
>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Michal Borowiecki <mi...@openbet.com>.
Also, I still feel that putting initializer on the first cogroup can 
mislead users into thinking the first stream is in some way special.

Just my 5c.
Michał

On 13/06/17 09:54, Michal Borowiecki wrote:
>
> Agree completely with the argument for serdes belonging in the same 
> place as the state store name, which is in the aggregate method.
>
> Cheers,
>
> Michał
>
>
> On 12/06/17 18:20, Xavier Léauté wrote:
>> I think we are discussing two separate things here, so it might be 
>> worth clarifying:
>>
>> 1) the position of the initializer with respect to the aggregators. 
>> If I understand correctly, Guozhang seems to think it is more natural 
>> to specify the initializer first, despite it not bearing any relation 
>> to the first aggregator. I can see the argument for specifying the 
>> initializer first, but I think it is debatable whether mixing it into 
>> the first cogroup call leads to a cleaner API or not.
>>
>> 2) where the serde should be defined (if necessary). Looking at our 
>> existing APIs in KGroupedStreams, we always offer two aggregate() 
>> methods. The first one takes the name of the store and associated 
>> aggregate value serde e.g. KGroupedStream.aggregate(Initializer<VR> 
>> initializer, Aggregator<? super K, ? super V, VR> aggregator, 
>> Serde<VR> aggValueSerde, String queryableStoreName)
>> The second one only takes a state store supplier, and does not 
>> specify any serde, e.g. KGroupedStream.aggregate(Initializer<VR> 
>> initializer, Aggregator<? super K, ? super V, VR> aggregator, final 
>> StateStoreSupplier<KeyValueStore> storeSupplier)
>> Presumably, when specifying a state store supplier it shouldn't be 
>> necessary to specify an aggregate value serde, since the provided 
>> statestore might not need to serialize the values (e.g. it may just 
>> keep them as regular objects in heap) or it may have its own 
>> internal serialization format.
>>
>> For consistency I think it would be valuable to preserve the same two 
>> aggregate methods for cogroup as well. Since the serde is only 
>> required in one of the two cases, I believe the serde has no place in 
>> the first cogroup() call and should only have to be specified as part 
>> of the aggregate() method that takes a state store name. In the case 
>> of a state store supplier, no serde would be necessary.
>>
>> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wangguoz@gmail.com 
>> <ma...@gmail.com>> wrote:
>>
>>     I'd agree that the aggregate value serde and the initializer does
>>     not bear direct relationship with the first `cogroup` calls, but
>>     after I tried to write some example code with these two different
>>     set of APIs I felt the current APIs just program more naturally.
>>
>>     I know it is kinda subjective, but I do think that user
>>     experience may be more important as a deciding factor than the
>>     logical argument for public interfaces. So I'd recommend people
>>     to also try out writing some example lines also and we can circle
>>     back and discuss which one feels more natural to write code.
>>
>>
>>     Guozhang
>>
>>     On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki
>>     <michal.borowiecki@openbet.com
>>     <ma...@openbet.com>> wrote:
>>
>>>         I feel it would make more sense to move the initializer and
>>>         serde to the
>>>         final aggregate statement, since the serde only applies to
>>>         the state store,
>>>         and the initializer doesn't bear any relation to the first
>>>         group in
>>>         particular.
>>         +1 for moving initializer and serde from cogroup() to the
>>         aggregate() for the reasons mentioned above.
>>
>>         Cheers,
>>
>>         Michał
>>
>>
>>         On 08/06/17 22:44, Guozhang Wang wrote:
>>
>>>         Note that although the internal `AbstractStoreSupplier` does maintain the
>>>         key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>>         to always retain that information, and hence we cannot assume that
>>>         StateStoreSuppliers always retain key / value serdes.
>>>
>>>         On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>  wrote:
>>>
>>>>         Another reason for the serde not to be in the first cogroup call, is that
>>>>         the serde should not be required if you pass a StateStoreSupplier to
>>>>         aggregate()
>>>>
>>>>         Regarding the aggregated type <T> I don't the why initializer should be
>>>>         favored over aggregator to define the type. In my mind separating the
>>>>         initializer into the last aggregate call clearly indicates that the
>>>>         initializer is independent of any of the aggregators or streams and that we
>>>>         don't wait for grouped1 events to initialize the co-group.
>>>>
>>>>         On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>  wrote:
>>>>
>>>>>         On a second thought... This is the current proposal API
>>>>>
>>>>>
>>>>>         ```
>>>>>
>>>>>         <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>>         final
>>>>>         Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>>>>         aggValueSerde)
>>>>>
>>>>>         ```
>>>>>
>>>>>
>>>>>         If we do not have the initializer in the first co-group it might be a bit
>>>>>         awkward for users to specify the aggregator that returns a typed <T>
>>>>         value?
>>>>>         Maybe it is still better to put these two functions in the same api?
>>>>>
>>>>>
>>>>>
>>>>>         Guozhang
>>>>>
>>>>>         On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>>         wrote:
>>>>>>         This suggestion lgtm. I would vote for the first alternative than
>>>>         adding
>>>>>>         it to the `KStreamBuilder` though.
>>>>>>
>>>>>>         On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>
>>>>>>         wrote:
>>>>>>
>>>>>>>         I have a minor suggestion to make the API a little bit more symmetric.
>>>>>>>         I feel it would make more sense to move the initializer and serde to
>>>>         the
>>>>>>>         final aggregate statement, since the serde only applies to the state
>>>>>>>         store,
>>>>>>>         and the initializer doesn't bear any relation to the first group in
>>>>>>>         particular. It would end up looking like this:
>>>>>>>
>>>>>>>         KTable<K, CG> cogrouped =
>>>>>>>              grouped1.cogroup(aggregator1)
>>>>>>>                      .cogroup(grouped2, aggregator2)
>>>>>>>                      .cogroup(grouped3, aggregator3)
>>>>>>>                      .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>>
>>>>>>>         Alternatively, we could move the the first cogroup() method to
>>>>>>>         KStreamBuilder, similar to how we have .merge()
>>>>>>>         and end up with an api that would be even more symmetric.
>>>>>>>
>>>>>>>         KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>>>>                        .cogroup(grouped2, aggregator2)
>>>>>>>                        .cogroup(grouped3, aggregator3)
>>>>>>>                        .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>>
>>>>>>>         This doesn't have to be a blocker, but I thought it would make the API
>>>>>>>         just
>>>>>>>         a tad cleaner.
>>>>>>>
>>>>>>>         On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>>>         wrote:
>>>>>>>>         Kyle,
>>>>>>>>
>>>>>>>>         Thanks a lot for the updated KIP. It looks good to me.
>>>>>>>>
>>>>>>>>
>>>>>>>>         Guozhang
>>>>>>>>
>>>>>>>>
>>>>>>>>         On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski<ji...@jagunet.com> <ma...@jagunet.com>
>>>>>         wrote:
>>>>>>>>>         This makes much more sense to me. +1
>>>>>>>>>
>>>>>>>>>>         On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>>         wrote:
>>>>>>>>>>         I have updated the KIP and my PR. Let me know what you think.
>>>>>>>>>>         To created a cogrouped stream just call cogroup on a
>>>>>         KgroupedStream
>>>>>>>         and
>>>>>>>>>>         supply the initializer, aggValueSerde, and an aggregator. Then
>>>>>>>         continue
>>>>>>>>>>         adding kgroupedstreams and aggregators. Then call one of the
>>>>         many
>>>>>>>>>         aggregate
>>>>>>>>>>         calls to create a KTable.
>>>>>>>>>>
>>>>>>>>>>         Thanks,
>>>>>>>>>>         Kyle
>>>>>>>>>>
>>>>>>>>>>         On Jun 1, 2017 4:03 AM, "Damian Guy"<da...@gmail.com> <ma...@gmail.com>
>>>>>         wrote:
>>>>>>>>>>>         Hi Kyle,
>>>>>>>>>>>
>>>>>>>>>>>         Thanks for the update. I think just one initializer makes sense
>>>>>         as
>>>>>>>         it
>>>>>>>>>>>         should only be called once per key and generally it is just
>>>>         going
>>>>>>>         to
>>>>>>>>>         create
>>>>>>>>>>>         a new instance of whatever the Aggregate class is.
>>>>>>>>>>>
>>>>>>>>>>>         Cheers,
>>>>>>>>>>>         Damian
>>>>>>>>>>>
>>>>>>>>>>>         On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>
>>>>>>>>>>>         wrote:
>>>>>>>>>>>
>>>>>>>>>>>>         Hello all,
>>>>>>>>>>>>
>>>>>>>>>>>>         I have spent some more time on this and the best alternative I
>>>>>>>         have
>>>>>>>>>         come
>>>>>>>>>>>         up
>>>>>>>>>>>>         with is:
>>>>>>>>>>>>         KGroupedStream has a single cogroup call that takes an
>>>>>         initializer
>>>>>>>>         and
>>>>>>>>>         an
>>>>>>>>>>>>         aggregator.
>>>>>>>>>>>>         CogroupedKStream has a cogroup call that takes additional
>>>>>>>>         groupedStream
>>>>>>>>>>>>         aggregator pairs.
>>>>>>>>>>>>         CogroupedKStream has multiple aggregate methods that create
>>>>         the
>>>>>>>>>         different
>>>>>>>>>>>>         stores.
>>>>>>>>>>>>
>>>>>>>>>>>>         I plan on updating the kip but I want people's input on if we
>>>>>>>         should
>>>>>>>>>         have
>>>>>>>>>>>>         the initializer be passed in once at the beginning or if we
>>>>>         should
>>>>>>>>>>>         instead
>>>>>>>>>>>>         have the initializer be required for each call to one of the
>>>>>>>>         aggregate
>>>>>>>>>>>>         calls. The first makes more sense to me but doesnt allow the
>>>>>         user
>>>>>>>         to
>>>>>>>>>>>>         specify different initializers for different tables.
>>>>>>>>>>>>
>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>
>>>>>>>>>>>>         On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>>         Yea I really like that idea I'll see what I can do to update
>>>>>         the
>>>>>>>         kip
>>>>>>>>>>>         and
>>>>>>>>>>>>>         my pr when I have some time. I'm not sure how well creating
>>>>         the
>>>>>>>>>>>>>         kstreamaggregates will go though because at that point I will
>>>>>>>         have
>>>>>>>>>>>         thrown
>>>>>>>>>>>>>         away the type of the values. It will be type safe I just may
>>>>>>>         need to
>>>>>>>>>>>         do a
>>>>>>>>>>>>>         little forcing.
>>>>>>>>>>>>>
>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>
>>>>>>>>>>>>>         On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>>         wrote:
>>>>>>>>>>>>>>         Kyle,
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Thanks for the explanations, my previous read on the wiki
>>>>>>>         examples
>>>>>>>>>         was
>>>>>>>>>>>>>>         wrong.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         So I guess my motivation should be "reduced" to: can we move
>>>>>         the
>>>>>>>>>>>         window
>>>>>>>>>>>>>>         specs param from "KGroupedStream#cogroup(..)" to
>>>>>>>>>>>>>>         "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         1. minor: we can reduce the #.generics in CogroupedKStream
>>>>>         from
>>>>>>>         3
>>>>>>>>         to
>>>>>>>>>>>         2.
>>>>>>>>>>>>>>         2. major: this is for extensibility of the APIs, and since
>>>>         we
>>>>>>>         are
>>>>>>>>>>>>         removing
>>>>>>>>>>>>>>         the "Evolving" annotations on Streams it may be harder to
>>>>>>>         change it
>>>>>>>>>>>>         again
>>>>>>>>>>>>>>         in the future. The extended use cases are that people wanted
>>>>>         to
>>>>>>>>         have
>>>>>>>>>>>>>>         windowed running aggregates on different granularities, e.g.
>>>>>>>         "give
>>>>>>>>         me
>>>>>>>>>>>>         the
>>>>>>>>>>>>>>         counts per-minute, per-hour, per-day and per-week", and
>>>>         today
>>>>>         in
>>>>>>>>         DSL
>>>>>>>>>>>         we
>>>>>>>>>>>>>>         need to specify that case in multiple aggregate operators,
>>>>>         which
>>>>>>>>         gets
>>>>>>>>>>>         a
>>>>>>>>>>>>>>         state store / changelog, etc. And it is possible to optimize
>>>>>         it
>>>>>>>         as
>>>>>>>>>>>         well
>>>>>>>>>>>>         to
>>>>>>>>>>>>>>         a single state store. Its implementation would be tricky as
>>>>>         you
>>>>>>>>         need
>>>>>>>>>>>         to
>>>>>>>>>>>>>>         contain different lengthed windows within your window store
>>>>>         but
>>>>>>>>         just
>>>>>>>>>>>>         from
>>>>>>>>>>>>>>         the public API point of view, it could be specified as:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>>>>>>>>>>>         "state-store-name");
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         table1 = stream.aggregate(/*per-minute window*/)
>>>>>>>>>>>>>>         table2 = stream.aggregate(/*per-hour window*/)
>>>>>>>>>>>>>>         table3 = stream.aggregate(/*per-day window*/)
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         while underlying we are only using a single store
>>>>>>>>         "state-store-name"
>>>>>>>>>>>         for
>>>>>>>>>>>>>>         it.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Although this feature is out of the scope of this KIP, I'd
>>>>>         like
>>>>>>>         to
>>>>>>>>>>>>         discuss
>>>>>>>>>>>>>>         if we can "leave the door open" to make such changes without
>>>>>>>>>         modifying
>>>>>>>>>>>>         the
>>>>>>>>>>>>>>         public APIs .
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         I allow defining a single window/sessionwindow one time
>>>>         when
>>>>>>>         you
>>>>>>>>>>>         make
>>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>         cogroup call from a KGroupedStream. From then on you are
>>>>>         using
>>>>>>>         the
>>>>>>>>>>>>>>         cogroup
>>>>>>>>>>>>>>>         call from with in CogroupedKStream which doesnt accept any
>>>>>>>>>>>         additional
>>>>>>>>>>>>>>>         windows/sessionwindows.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         Is this what you meant by your question or did I
>>>>>         misunderstand?
>>>>>>>>>>>>>>>         On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>>         wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>         Another question that came to me is on "window alignment":
>>>>>         from
>>>>>>>>         the
>>>>>>>>>>>>         KIP
>>>>>>>>>>>>>>         it
>>>>>>>>>>>>>>>         seems you are allowing users to specify a (potentially
>>>>>>>         different)
>>>>>>>>>>>>         window
>>>>>>>>>>>>>>>         spec in each co-grouped input stream. So if these window
>>>>>         specs
>>>>>>>         are
>>>>>>>>>>>>>>>         different how should we "align" them with different input
>>>>>>>>         streams? I
>>>>>>>>>>>>>>         think
>>>>>>>>>>>>>>>         it is more natural to only specify on window spec in the
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         And remove it from the cogroup() functions. WDYT?
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>>>>>         wangguoz@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>         Thanks for the proposal Kyle, this is a quite common use
>>>>>         case
>>>>>>>         to
>>>>>>>>>>>>>>         support
>>>>>>>>>>>>>>>>         such multi-way table join (i.e. N source tables with N
>>>>>>>         aggregate
>>>>>>>>>>>>         func)
>>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>>         a single store and N+1 serdes, I have seen lots of people
>>>>>>>         using
>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>         low-level PAPI to achieve this goal.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>         I like your point about not handling other cases such as
>>>>>>>         count
>>>>>>>>>>>         and
>>>>>>>>>>>>>>>         reduce.
>>>>>>>>>>>>>>>>>         I think that reduce may not make sense because reduce
>>>>>         assumes
>>>>>>>>>>>         that
>>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>>         input values are the same as the output values. With
>>>>>         cogroup
>>>>>>>>>>>         there
>>>>>>>>>>>>>>         may
>>>>>>>>>>>>>>>         be
>>>>>>>>>>>>>>>>>         multiple different input types and then your output type
>>>>>>>         cant be
>>>>>>>>>>>>>>>         multiple
>>>>>>>>>>>>>>>>>         different things. In the case where you have all matching
>>>>>>>         value
>>>>>>>>>>>>         types
>>>>>>>>>>>>>>>         you
>>>>>>>>>>>>>>>>>         can do KStreamBuilder#merge followed by the reduce.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         As for count I think it is possible to call count on all
>>>>>         the
>>>>>>>>>>>>>>         individual
>>>>>>>>>>>>>>>>>         grouped streams and then do joins. Otherwise we could
>>>>         maybe
>>>>>>>         make
>>>>>>>>>>>         a
>>>>>>>>>>>>>>>         special
>>>>>>>>>>>>>>>>>         call in groupedstream for this case. Because in this case
>>>>>         we
>>>>>>>>         dont
>>>>>>>>>>>>>>         need
>>>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>>         do type checking on the values. It could be similar to
>>>>         the
>>>>>>>>>>>         current
>>>>>>>>>>>>>>         count
>>>>>>>>>>>>>>>>>         methods but accept a var args of additonal grouped
>>>>         streams
>>>>>         as
>>>>>>>>>>>         well
>>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>>         make
>>>>>>>>>>>>>>>>>         sure they have a key type of K.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         The way I have put the kip together is to ensure that we
>>>>         do
>>>>>>>         type
>>>>>>>>>>>>>>>         checking.
>>>>>>>>>>>>>>>>>         I don't see a way we could group them all first and then
>>>>>>>         make a
>>>>>>>>>>>>         call
>>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>>         count, reduce, or aggregate because with aggregate they
>>>>>         would
>>>>>>>>>>>         need
>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>         pass
>>>>>>>>>>>>>>>>>         a list of aggregators and we would have no way of type
>>>>>>>         checking
>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>         they
>>>>>>>>>>>>>>>>>         match the grouped streams.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         On May 19, 2017 11:42 AM, "Xavier Léauté" <
>>>>>>>         xavier@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>>         Sorry to jump on this thread so late. I agree this is a
>>>>>         very
>>>>>>>>>>>>         useful
>>>>>>>>>>>>>>>>>>         addition and wanted to provide an additional use-case
>>>>         and
>>>>>>>         some
>>>>>>>>>>>>         more
>>>>>>>>>>>>>>>>>>         comments.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         This is actually a very common analytics use-case in the
>>>>>>>>>>>         ad-tech
>>>>>>>>>>>>>>>>>         industry.
>>>>>>>>>>>>>>>>>>         The typical setup will have an auction stream, an
>>>>>         impression
>>>>>>>>>>>>>>         stream,
>>>>>>>>>>>>>>>>>         and a
>>>>>>>>>>>>>>>>>>         click stream. Those three streams need to be combined to
>>>>>>>>>>>         compute
>>>>>>>>>>>>>>>>>         aggregate
>>>>>>>>>>>>>>>>>>         statistics (e.g. impression statistics, and
>>>>         click-through
>>>>>>>>>>>         rates),
>>>>>>>>>>>>>>>         since
>>>>>>>>>>>>>>>>>>         most of the attributes of interest are only present the
>>>>>>>         auction
>>>>>>>>>>>>>>>         stream.
>>>>>>>>>>>>>>>>>>         A simple way to do this is to co-group all the streams
>>>>         by
>>>>>>>         the
>>>>>>>>>>>>>>         auction
>>>>>>>>>>>>>>>>>         key,
>>>>>>>>>>>>>>>>>>         and process updates to the co-group as events for each
>>>>>>>         stream
>>>>>>>>>>>>         come
>>>>>>>>>>>>>>         in,
>>>>>>>>>>>>>>>>>>         keeping only one value from each stream before sending
>>>>>>>>>>>         downstream
>>>>>>>>>>>>>>         for
>>>>>>>>>>>>>>>>>>         further processing / aggregation.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         One could view the result of that co-group operation as
>>>>         a
>>>>>>>>>>>>         "KTable"
>>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>>>>         multiple values per key. The key being the grouping key,
>>>>>         and
>>>>>>>>>>>         the
>>>>>>>>>>>>>>>         values
>>>>>>>>>>>>>>>>>>         consisting of one value per stream.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         What I like about Kyle's approach is that allows elegant
>>>>>>>>>>>>>>         co-grouping
>>>>>>>>>>>>>>>         of
>>>>>>>>>>>>>>>>>>         multiple streams without having to worry about the
>>>>         number
>>>>>         of
>>>>>>>>>>>>>>         streams,
>>>>>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>>>         avoids dealing with Tuple types or other generic
>>>>>         interfaces
>>>>>>>>>>>         that
>>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>>         get
>>>>>>>>>>>>>>>>>>         messy if we wanted to preserve all the value types in
>>>>         the
>>>>>>>>>>>>         resulting
>>>>>>>>>>>>>>>>>>         co-grouped stream.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         My only concern is that we only allow the cogroup +
>>>>>>>         aggregate
>>>>>>>>>>>>>>         combined
>>>>>>>>>>>>>>>>>>         operation. This forces the user to build their own tuple
>>>>>>>>>>>>>>         serialization
>>>>>>>>>>>>>>>>>>         format if they want to preserve the individual input
>>>>>         stream
>>>>>>>>>>>>         values
>>>>>>>>>>>>>>         as
>>>>>>>>>>>>>>>         a
>>>>>>>>>>>>>>>>>>         group. It also deviates quite a bit from our approach in
>>>>>>>>>>>>>>>         KGroupedStream
>>>>>>>>>>>>>>>>>>         which offers other operations, such as count and reduce,
>>>>>>>         which
>>>>>>>>>>>>>>         should
>>>>>>>>>>>>>>>>>         also
>>>>>>>>>>>>>>>>>>         be applicable to a co-grouped stream.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         Overall I still think this is a really useful addition,
>>>>>         but
>>>>>>>         I
>>>>>>>>>>>>         feel
>>>>>>>>>>>>>>         we
>>>>>>>>>>>>>>>>>>         haven't spend much time trying to explore alternative
>>>>         DSLs
>>>>>>>         that
>>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>>         maybe
>>>>>>>>>>>>>>>>>>         generalize better or match our existing syntax more
>>>>>         closely.
>>>>>>>>>>>>>>>>>>         On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         Eno, is there anyone else that is an expert in the
>>
>>>>         kafka
>>>>>>>>>>>>         streams
>>>>>>>>>>>>>>>         realm
>>>>>>>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>>>>>         I should reach out to for input?
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>         I believe Damian Guy is still planning on reviewing
>>>>         this
>>>>>>>         more
>>>>>>>>>>>>         in
>>>>>>>>>>>>>>>         depth
>>>>>>>>>>>>>>>>>>         so I
>>
>
> -- 
> Signature
> <http://www.openbet.com/> 	Michal Borowiecki
> Senior Software Engineer L4
> 	T: 	+44 208 742 1600
>
> 	
> 	+44 203 249 8448
>
> 	
> 	
> 	E: 	michal.borowiecki@openbet.com
> 	W: 	www.openbet.com <http://www.openbet.com/>
>
> 	
> 	OpenBet Ltd
>
> 	Chiswick Park Building 9
>
> 	566 Chiswick High Rd
>
> 	London
>
> 	W4 5XT
>
> 	UK
>
> 	
> <https://www.openbet.com/email_promo>
>
> This message is confidential and intended only for the addressee. If 
> you have received this message in error, please immediately notify the 
> postmaster@openbet.com <ma...@openbet.com> and delete it 
> from your system as well as any copies. The content of e-mails as well 
> as traffic data may be monitored by OpenBet for employment and 
> security purposes. To protect the environment please do not print this 
> e-mail unless necessary. OpenBet Ltd. Registered Office: Chiswick Park 
> Building 9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A 
> company registered in England and Wales. Registered no. 3134634. VAT 
> no. GB927523612
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Michal Borowiecki <mi...@openbet.com>.
Agree completely with the argument for serdes belonging in the same 
place as the state store name, which is in the aggregate method.

Cheers,

Michał


On 12/06/17 18:20, Xavier Léauté wrote:
> I think we are discussing two separate things here, so it might be 
> worth clarifying:
>
> 1) the position of the initializer with respect to the aggregators. If 
> I understand correctly, Guozhang seems to think it is more natural to 
> specify the initializer first, despite it not bearing any relation to 
> the first aggregator. I can see the argument for specifying the 
> initializer first, but I think it is debatable whether mixing it into 
> the first cogroup call leads to a cleaner API or not.
>
> 2) where the serde should be defined (if necessary). Looking at our 
> existing APIs in KGroupedStreams, we always offer two aggregate() 
> methods. The first one takes the name of the store and associated 
> aggregate value serde e.g. KGroupedStream.aggregate(Initializer<VR> 
> initializer, Aggregator<? super K, ? super V, VR> aggregator, 
> Serde<VR> aggValueSerde, String queryableStoreName)
> The second one only takes a state store supplier, and does not specify 
> any serde, e.g. KGroupedStream.aggregate(Initializer<VR> 
> initializer, Aggregator<? super K, ? super V, VR> aggregator, final 
> StateStoreSupplier<KeyValueStore> storeSupplier)
> Presumably, when specifying a state store supplier it shouldn't be 
> necessary to specify an aggregate value serde, since the provided 
> statestore might not need to serialize the values (e.g. it may just 
> keep them as regular objects in heap) or it may have its own 
> internal serialization format.
>
> For consistency I think it would be valuable to preserve the same two 
> aggregate methods for cogroup as well. Since the serde is only 
> required in one of the two cases, I believe the serde has no place in 
> the first cogroup() call and should only have to be specified as part 
> of the aggregate() method that takes a state store name. In the case 
> of a state store supplier, no serde would be necessary.
>
> On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wangguoz@gmail.com 
> <ma...@gmail.com>> wrote:
>
>     I'd agree that the aggregate value serde and the initializer does
>     not bear direct relationship with the first `cogroup` calls, but
>     after I tried to write some example code with these two different
>     set of APIs I felt the current APIs just program more naturally.
>
>     I know it is kinda subjective, but I do think that user experience
>     may be more important as a deciding factor than the logical
>     argument for public interfaces. So I'd recommend people to also
>     try out writing some example lines also and we can circle back and
>     discuss which one feels more natural to write code.
>
>
>     Guozhang
>
>     On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki
>     <michal.borowiecki@openbet.com
>     <ma...@openbet.com>> wrote:
>
>>         I feel it would make more sense to move the initializer and
>>         serde to the
>>         final aggregate statement, since the serde only applies to
>>         the state store,
>>         and the initializer doesn't bear any relation to the first
>>         group in
>>         particular.
>         +1 for moving initializer and serde from cogroup() to the
>         aggregate() for the reasons mentioned above.
>
>         Cheers,
>
>         Michał
>
>
>         On 08/06/17 22:44, Guozhang Wang wrote:
>
>>         Note that although the internal `AbstractStoreSupplier` does maintain the
>>         key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>>         to always retain that information, and hence we cannot assume that
>>         StateStoreSuppliers always retain key / value serdes.
>>
>>         On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>  wrote:
>>
>>>         Another reason for the serde not to be in the first cogroup call, is that
>>>         the serde should not be required if you pass a StateStoreSupplier to
>>>         aggregate()
>>>
>>>         Regarding the aggregated type <T> I don't the why initializer should be
>>>         favored over aggregator to define the type. In my mind separating the
>>>         initializer into the last aggregate call clearly indicates that the
>>>         initializer is independent of any of the aggregators or streams and that we
>>>         don't wait for grouped1 events to initialize the co-group.
>>>
>>>         On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>  wrote:
>>>
>>>>         On a second thought... This is the current proposal API
>>>>
>>>>
>>>>         ```
>>>>
>>>>         <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>>         final
>>>>         Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>>>         aggValueSerde)
>>>>
>>>>         ```
>>>>
>>>>
>>>>         If we do not have the initializer in the first co-group it might be a bit
>>>>         awkward for users to specify the aggregator that returns a typed <T>
>>>         value?
>>>>         Maybe it is still better to put these two functions in the same api?
>>>>
>>>>
>>>>
>>>>         Guozhang
>>>>
>>>>         On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>         wrote:
>>>>>         This suggestion lgtm. I would vote for the first alternative than
>>>         adding
>>>>>         it to the `KStreamBuilder` though.
>>>>>
>>>>>         On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté<xa...@confluent.io> <ma...@confluent.io>
>>>>>         wrote:
>>>>>
>>>>>>         I have a minor suggestion to make the API a little bit more symmetric.
>>>>>>         I feel it would make more sense to move the initializer and serde to
>>>         the
>>>>>>         final aggregate statement, since the serde only applies to the state
>>>>>>         store,
>>>>>>         and the initializer doesn't bear any relation to the first group in
>>>>>>         particular. It would end up looking like this:
>>>>>>
>>>>>>         KTable<K, CG> cogrouped =
>>>>>>              grouped1.cogroup(aggregator1)
>>>>>>                      .cogroup(grouped2, aggregator2)
>>>>>>                      .cogroup(grouped3, aggregator3)
>>>>>>                      .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>
>>>>>>         Alternatively, we could move the the first cogroup() method to
>>>>>>         KStreamBuilder, similar to how we have .merge()
>>>>>>         and end up with an api that would be even more symmetric.
>>>>>>
>>>>>>         KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>>>                        .cogroup(grouped2, aggregator2)
>>>>>>                        .cogroup(grouped3, aggregator3)
>>>>>>                        .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>>
>>>>>>         This doesn't have to be a blocker, but I thought it would make the API
>>>>>>         just
>>>>>>         a tad cleaner.
>>>>>>
>>>>>>         On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang<wa...@gmail.com> <ma...@gmail.com>
>>>>         wrote:
>>>>>>>         Kyle,
>>>>>>>
>>>>>>>         Thanks a lot for the updated KIP. It looks good to me.
>>>>>>>
>>>>>>>
>>>>>>>         Guozhang
>>>>>>>
>>>>>>>
>>>>>>>         On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski<ji...@jagunet.com> <ma...@jagunet.com>
>>>>         wrote:
>>>>>>>>         This makes much more sense to me. +1
>>>>>>>>
>>>>>>>>>         On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>         wrote:
>>>>>>>>>         I have updated the KIP and my PR. Let me know what you think.
>>>>>>>>>         To created a cogrouped stream just call cogroup on a
>>>>         KgroupedStream
>>>>>>         and
>>>>>>>>>         supply the initializer, aggValueSerde, and an aggregator. Then
>>>>>>         continue
>>>>>>>>>         adding kgroupedstreams and aggregators. Then call one of the
>>>         many
>>>>>>>>         aggregate
>>>>>>>>>         calls to create a KTable.
>>>>>>>>>
>>>>>>>>>         Thanks,
>>>>>>>>>         Kyle
>>>>>>>>>
>>>>>>>>>         On Jun 1, 2017 4:03 AM, "Damian Guy"<da...@gmail.com> <ma...@gmail.com>
>>>>         wrote:
>>>>>>>>>>         Hi Kyle,
>>>>>>>>>>
>>>>>>>>>>         Thanks for the update. I think just one initializer makes sense
>>>>         as
>>>>>>         it
>>>>>>>>>>         should only be called once per key and generally it is just
>>>         going
>>>>>>         to
>>>>>>>>         create
>>>>>>>>>>         a new instance of whatever the Aggregate class is.
>>>>>>>>>>
>>>>>>>>>>         Cheers,
>>>>>>>>>>         Damian
>>>>>>>>>>
>>>>>>>>>>         On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>
>>>>>>>>>>         wrote:
>>>>>>>>>>
>>>>>>>>>>>         Hello all,
>>>>>>>>>>>
>>>>>>>>>>>         I have spent some more time on this and the best alternative I
>>>>>>         have
>>>>>>>>         come
>>>>>>>>>>         up
>>>>>>>>>>>         with is:
>>>>>>>>>>>         KGroupedStream has a single cogroup call that takes an
>>>>         initializer
>>>>>>>         and
>>>>>>>>         an
>>>>>>>>>>>         aggregator.
>>>>>>>>>>>         CogroupedKStream has a cogroup call that takes additional
>>>>>>>         groupedStream
>>>>>>>>>>>         aggregator pairs.
>>>>>>>>>>>         CogroupedKStream has multiple aggregate methods that create
>>>         the
>>>>>>>>         different
>>>>>>>>>>>         stores.
>>>>>>>>>>>
>>>>>>>>>>>         I plan on updating the kip but I want people's input on if we
>>>>>>         should
>>>>>>>>         have
>>>>>>>>>>>         the initializer be passed in once at the beginning or if we
>>>>         should
>>>>>>>>>>         instead
>>>>>>>>>>>         have the initializer be required for each call to one of the
>>>>>>>         aggregate
>>>>>>>>>>>         calls. The first makes more sense to me but doesnt allow the
>>>>         user
>>>>>>         to
>>>>>>>>>>>         specify different initializers for different tables.
>>>>>>>>>>>
>>>>>>>>>>>         Thanks,
>>>>>>>>>>>         Kyle
>>>>>>>>>>>
>>>>>>>>>>>         On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>>>         winkelman.kyle@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>         wrote:
>>>>>>>>>>>
>>>>>>>>>>>>         Yea I really like that idea I'll see what I can do to update
>>>>         the
>>>>>>         kip
>>>>>>>>>>         and
>>>>>>>>>>>>         my pr when I have some time. I'm not sure how well creating
>>>         the
>>>>>>>>>>>>         kstreamaggregates will go though because at that point I will
>>>>>>         have
>>>>>>>>>>         thrown
>>>>>>>>>>>>         away the type of the values. It will be type safe I just may
>>>>>>         need to
>>>>>>>>>>         do a
>>>>>>>>>>>>         little forcing.
>>>>>>>>>>>>
>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>
>>>>>>>>>>>>         On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>         wrote:
>>>>>>>>>>>>>         Kyle,
>>>>>>>>>>>>>
>>>>>>>>>>>>>         Thanks for the explanations, my previous read on the wiki
>>>>>>         examples
>>>>>>>>         was
>>>>>>>>>>>>>         wrong.
>>>>>>>>>>>>>
>>>>>>>>>>>>>         So I guess my motivation should be "reduced" to: can we move
>>>>         the
>>>>>>>>>>         window
>>>>>>>>>>>>>         specs param from "KGroupedStream#cogroup(..)" to
>>>>>>>>>>>>>         "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>>>>>>>>>>
>>>>>>>>>>>>>         1. minor: we can reduce the #.generics in CogroupedKStream
>>>>         from
>>>>>>         3
>>>>>>>         to
>>>>>>>>>>         2.
>>>>>>>>>>>>>         2. major: this is for extensibility of the APIs, and since
>>>         we
>>>>>>         are
>>>>>>>>>>>         removing
>>>>>>>>>>>>>         the "Evolving" annotations on Streams it may be harder to
>>>>>>         change it
>>>>>>>>>>>         again
>>>>>>>>>>>>>         in the future. The extended use cases are that people wanted
>>>>         to
>>>>>>>         have
>>>>>>>>>>>>>         windowed running aggregates on different granularities, e.g.
>>>>>>         "give
>>>>>>>         me
>>>>>>>>>>>         the
>>>>>>>>>>>>>         counts per-minute, per-hour, per-day and per-week", and
>>>         today
>>>>         in
>>>>>>>         DSL
>>>>>>>>>>         we
>>>>>>>>>>>>>         need to specify that case in multiple aggregate operators,
>>>>         which
>>>>>>>         gets
>>>>>>>>>>         a
>>>>>>>>>>>>>         state store / changelog, etc. And it is possible to optimize
>>>>         it
>>>>>>         as
>>>>>>>>>>         well
>>>>>>>>>>>         to
>>>>>>>>>>>>>         a single state store. Its implementation would be tricky as
>>>>         you
>>>>>>>         need
>>>>>>>>>>         to
>>>>>>>>>>>>>         contain different lengthed windows within your window store
>>>>         but
>>>>>>>         just
>>>>>>>>>>>         from
>>>>>>>>>>>>>         the public API point of view, it could be specified as:
>>>>>>>>>>>>>
>>>>>>>>>>>>>         CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>>>>>>>>>>         "state-store-name");
>>>>>>>>>>>>>
>>>>>>>>>>>>>         table1 = stream.aggregate(/*per-minute window*/)
>>>>>>>>>>>>>         table2 = stream.aggregate(/*per-hour window*/)
>>>>>>>>>>>>>         table3 = stream.aggregate(/*per-day window*/)
>>>>>>>>>>>>>
>>>>>>>>>>>>>         while underlying we are only using a single store
>>>>>>>         "state-store-name"
>>>>>>>>>>         for
>>>>>>>>>>>>>         it.
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>         Although this feature is out of the scope of this KIP, I'd
>>>>         like
>>>>>>         to
>>>>>>>>>>>         discuss
>>>>>>>>>>>>>         if we can "leave the door open" to make such changes without
>>>>>>>>         modifying
>>>>>>>>>>>         the
>>>>>>>>>>>>>         public APIs .
>>>>>>>>>>>>>
>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>         On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>
>>>>>>>>>>>>>>         I allow defining a single window/sessionwindow one time
>>>         when
>>>>>>         you
>>>>>>>>>>         make
>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>         cogroup call from a KGroupedStream. From then on you are
>>>>         using
>>>>>>         the
>>>>>>>>>>>>>         cogroup
>>>>>>>>>>>>>>         call from with in CogroupedKStream which doesnt accept any
>>>>>>>>>>         additional
>>>>>>>>>>>>>>         windows/sessionwindows.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Is this what you meant by your question or did I
>>>>         misunderstand?
>>>>>>>>>>>>>>         On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>>         wangguoz@gmail.com <ma...@gmail.com>
>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>         Another question that came to me is on "window alignment":
>>>>         from
>>>>>>>         the
>>>>>>>>>>>         KIP
>>>>>>>>>>>>>         it
>>>>>>>>>>>>>>         seems you are allowing users to specify a (potentially
>>>>>>         different)
>>>>>>>>>>>         window
>>>>>>>>>>>>>>         spec in each co-grouped input stream. So if these window
>>>>         specs
>>>>>>         are
>>>>>>>>>>>>>>         different how should we "align" them with different input
>>>>>>>         streams? I
>>>>>>>>>>>>>         think
>>>>>>>>>>>>>>         it is more natural to only specify on window spec in the
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         And remove it from the cogroup() functions. WDYT?
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>         On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>>>>         wangguoz@gmail.com <ma...@gmail.com>>
>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>         Thanks for the proposal Kyle, this is a quite common use
>>>>         case
>>>>>>         to
>>>>>>>>>>>>>         support
>>>>>>>>>>>>>>>         such multi-way table join (i.e. N source tables with N
>>>>>>         aggregate
>>>>>>>>>>>         func)
>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>         a single store and N+1 serdes, I have seen lots of people
>>>>>>         using
>>>>>>>>>>         the
>>>>>>>>>>>>>>>         low-level PAPI to achieve this goal.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>         On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>         I like your point about not handling other cases such as
>>>>>>         count
>>>>>>>>>>         and
>>>>>>>>>>>>>>         reduce.
>>>>>>>>>>>>>>>>         I think that reduce may not make sense because reduce
>>>>         assumes
>>>>>>>>>>         that
>>>>>>>>>>>>>         the
>>>>>>>>>>>>>>>>         input values are the same as the output values. With
>>>>         cogroup
>>>>>>>>>>         there
>>>>>>>>>>>>>         may
>>>>>>>>>>>>>>         be
>>>>>>>>>>>>>>>>         multiple different input types and then your output type
>>>>>>         cant be
>>>>>>>>>>>>>>         multiple
>>>>>>>>>>>>>>>>         different things. In the case where you have all matching
>>>>>>         value
>>>>>>>>>>>         types
>>>>>>>>>>>>>>         you
>>>>>>>>>>>>>>>>         can do KStreamBuilder#merge followed by the reduce.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         As for count I think it is possible to call count on all
>>>>         the
>>>>>>>>>>>>>         individual
>>>>>>>>>>>>>>>>         grouped streams and then do joins. Otherwise we could
>>>         maybe
>>>>>>         make
>>>>>>>>>>         a
>>>>>>>>>>>>>>         special
>>>>>>>>>>>>>>>>         call in groupedstream for this case. Because in this case
>>>>         we
>>>>>>>         dont
>>>>>>>>>>>>>         need
>>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>         do type checking on the values. It could be similar to
>>>         the
>>>>>>>>>>         current
>>>>>>>>>>>>>         count
>>>>>>>>>>>>>>>>         methods but accept a var args of additonal grouped
>>>         streams
>>>>         as
>>>>>>>>>>         well
>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>         make
>>>>>>>>>>>>>>>>         sure they have a key type of K.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         The way I have put the kip together is to ensure that we
>>>         do
>>>>>>         type
>>>>>>>>>>>>>>         checking.
>>>>>>>>>>>>>>>>         I don't see a way we could group them all first and then
>>>>>>         make a
>>>>>>>>>>>         call
>>>>>>>>>>>>>         to
>>>>>>>>>>>>>>>>         count, reduce, or aggregate because with aggregate they
>>>>         would
>>>>>>>>>>         need
>>>>>>>>>>>         to
>>>>>>>>>>>>>>         pass
>>>>>>>>>>>>>>>>         a list of aggregators and we would have no way of type
>>>>>>         checking
>>>>>>>>>>>         that
>>>>>>>>>>>>>>         they
>>>>>>>>>>>>>>>>         match the grouped streams.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         Thanks,
>>>>>>>>>>>>>>>>         Kyle
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>         On May 19, 2017 11:42 AM, "Xavier Léauté" <
>>>>>>         xavier@confluent.io <ma...@confluent.io>>
>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>         Sorry to jump on this thread so late. I agree this is a
>>>>         very
>>>>>>>>>>>         useful
>>>>>>>>>>>>>>>>>         addition and wanted to provide an additional use-case
>>>         and
>>>>>>         some
>>>>>>>>>>>         more
>>>>>>>>>>>>>>>>>         comments.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         This is actually a very common analytics use-case in the
>>>>>>>>>>         ad-tech
>>>>>>>>>>>>>>>>         industry.
>>>>>>>>>>>>>>>>>         The typical setup will have an auction stream, an
>>>>         impression
>>>>>>>>>>>>>         stream,
>>>>>>>>>>>>>>>>         and a
>>>>>>>>>>>>>>>>>         click stream. Those three streams need to be combined to
>>>>>>>>>>         compute
>>>>>>>>>>>>>>>>         aggregate
>>>>>>>>>>>>>>>>>         statistics (e.g. impression statistics, and
>>>         click-through
>>>>>>>>>>         rates),
>>>>>>>>>>>>>>         since
>>>>>>>>>>>>>>>>>         most of the attributes of interest are only present the
>>>>>>         auction
>>>>>>>>>>>>>>         stream.
>>>>>>>>>>>>>>>>>         A simple way to do this is to co-group all the streams
>>>         by
>>>>>>         the
>>>>>>>>>>>>>         auction
>>>>>>>>>>>>>>>>         key,
>>>>>>>>>>>>>>>>>         and process updates to the co-group as events for each
>>>>>>         stream
>>>>>>>>>>>         come
>>>>>>>>>>>>>         in,
>>>>>>>>>>>>>>>>>         keeping only one value from each stream before sending
>>>>>>>>>>         downstream
>>>>>>>>>>>>>         for
>>>>>>>>>>>>>>>>>         further processing / aggregation.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         One could view the result of that co-group operation as
>>>         a
>>>>>>>>>>>         "KTable"
>>>>>>>>>>>>>>         with
>>>>>>>>>>>>>>>>>         multiple values per key. The key being the grouping key,
>>>>         and
>>>>>>>>>>         the
>>>>>>>>>>>>>>         values
>>>>>>>>>>>>>>>>>         consisting of one value per stream.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         What I like about Kyle's approach is that allows elegant
>>>>>>>>>>>>>         co-grouping
>>>>>>>>>>>>>>         of
>>>>>>>>>>>>>>>>>         multiple streams without having to worry about the
>>>         number
>>>>         of
>>>>>>>>>>>>>         streams,
>>>>>>>>>>>>>>>>         and
>>>>>>>>>>>>>>>>>         avoids dealing with Tuple types or other generic
>>>>         interfaces
>>>>>>>>>>         that
>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>         get
>>>>>>>>>>>>>>>>>         messy if we wanted to preserve all the value types in
>>>         the
>>>>>>>>>>>         resulting
>>>>>>>>>>>>>>>>>         co-grouped stream.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         My only concern is that we only allow the cogroup +
>>>>>>         aggregate
>>>>>>>>>>>>>         combined
>>>>>>>>>>>>>>>>>         operation. This forces the user to build their own tuple
>>>>>>>>>>>>>         serialization
>>>>>>>>>>>>>>>>>         format if they want to preserve the individual input
>>>>         stream
>>>>>>>>>>>         values
>>>>>>>>>>>>>         as
>>>>>>>>>>>>>>         a
>>>>>>>>>>>>>>>>>         group. It also deviates quite a bit from our approach in
>>>>>>>>>>>>>>         KGroupedStream
>>>>>>>>>>>>>>>>>         which offers other operations, such as count and reduce,
>>>>>>         which
>>>>>>>>>>>>>         should
>>>>>>>>>>>>>>>>         also
>>>>>>>>>>>>>>>>>         be applicable to a co-grouped stream.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>         Overall I still think this is a really useful addition,
>>>>         but
>>>>>>         I
>>>>>>>>>>>         feel
>>>>>>>>>>>>>         we
>>>>>>>>>>>>>>>>>         haven't spend much time trying to explore alternative
>>>         DSLs
>>>>>>         that
>>>>>>>>>>>>>         could
>>>>>>>>>>>>>>>>         maybe
>>>>>>>>>>>>>>>>>         generalize better or match our existing syntax more
>>>>         closely.
>>>>>>>>>>>>>>>>>         On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>>>>>>>>>>>>>         winkelman.kyle@gmail.com
>>>>>>>>>>>>>>         <ma...@gmail.com>
>>>>>>>>>>>>>>>>>         wrote:
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         Eno, is there anyone else that is an expert in the
>
>>>         kafka
>>>>>>>>>>>         streams
>>>>>>>>>>>>>>         realm
>>>>>>>>>>>>>>>>>         that
>>>>>>>>>>>>>>>>>>         I should reach out to for input?
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>         I believe Damian Guy is still planning on reviewing
>>>         this
>>>>>>         more
>>>>>>>>>>>         in
>>>>>>>>>>>>>>         depth
>>>>>>>>>>>>>>>>>         so I
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Xavier Léauté <xa...@confluent.io>.
I think we are discussing two separate things here, so it might be worth
clarifying:

1) the position of the initializer with respect to the aggregators. If I
understand correctly, Guozhang seems to think it is more natural to specify
the initializer first, despite it not bearing any relation to the first
aggregator. I can see the argument for specifying the initializer first,
but I think it is debatable whether mixing it into the first cogroup call
leads to a cleaner API or not.

2) where the serde should be defined (if necessary). Looking at our
existing APIs in KGroupedStreams, we always offer two aggregate() methods.
The first one takes the name of the store and associated aggregate value
serde e.g. KGroupedStream.aggregate(Initializer<VR> initializer,
Aggregator<? super K, ? super V, VR> aggregator, Serde<VR> aggValueSerde,
String queryableStoreName)
The second one only takes a state store supplier, and does not specify any
serde, e.g. KGroupedStream.aggregate(Initializer<VR>
initializer, Aggregator<? super K, ? super V, VR> aggregator, final
StateStoreSupplier<KeyValueStore> storeSupplier)
Presumably, when specifying a state store supplier it shouldn't be
necessary to specify an aggregate value serde, since the provided
statestore might not need to serialize the values (e.g. it may just keep
them as regular objects in heap) or it may have its own
internal serialization format.

For consistency I think it would be valuable to preserve the same two
aggregate methods for cogroup as well. Since the serde is only required in
one of the two cases, I believe the serde has no place in the first
cogroup() call and should only have to be specified as part of the
aggregate() method that takes a state store name. In the case of a state
store supplier, no serde would be necessary.


On Sat, Jun 10, 2017 at 4:09 PM Guozhang Wang <wa...@gmail.com> wrote:

> I'd agree that the aggregate value serde and the initializer does not bear
> direct relationship with the first `cogroup` calls, but after I tried to
> write some example code with these two different set of APIs I felt the
> current APIs just program more naturally.
>
> I know it is kinda subjective, but I do think that user experience may be
> more important as a deciding factor than the logical argument for public
> interfaces. So I'd recommend people to also try out writing some example
> lines also and we can circle back and discuss which one feels more natural
> to write code.
>
>
> Guozhang
>
> On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
> michal.borowiecki@openbet.com> wrote:
>
>> I feel it would make more sense to move the initializer and serde to the
>> final aggregate statement, since the serde only applies to the state
>> store,
>> and the initializer doesn't bear any relation to the first group in
>> particular.
>>
>> +1 for moving initializer and serde from cogroup() to the aggregate() for
>> the reasons mentioned above.
>>
>> Cheers,
>>
>> Michał
>>
>> On 08/06/17 22:44, Guozhang Wang wrote:
>>
> Note that although the internal `AbstractStoreSupplier` does maintain the
>> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
>> to always retain that information, and hence we cannot assume that
>> StateStoreSuppliers always retain key / value serdes.
>>
>> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io> wrote:
>>
>>
>> Another reason for the serde not to be in the first cogroup call, is that
>> the serde should not be required if you pass a StateStoreSupplier to
>> aggregate()
>>
>> Regarding the aggregated type <T> I don't the why initializer should be
>> favored over aggregator to define the type. In my mind separating the
>> initializer into the last aggregate call clearly indicates that the
>> initializer is independent of any of the aggregators or streams and that we
>> don't wait for grouped1 events to initialize the co-group.
>>
>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com> wrote:
>>
>>
>> On a second thought... This is the current proposal API
>>
>>
>> ```
>>
>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>>
>> final
>>
>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>> aggValueSerde)
>>
>> ```
>>
>>
>> If we do not have the initializer in the first co-group it might be a bit
>> awkward for users to specify the aggregator that returns a typed <T>
>>
>> value?
>>
>> Maybe it is still better to put these two functions in the same api?
>>
>>
>>
>> Guozhang
>>
>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>
>> wrote:
>>
>> This suggestion lgtm. I would vote for the first alternative than
>>
>> adding
>>
>> it to the `KStreamBuilder` though.
>>
>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io>
>> wrote:
>>
>>
>> I have a minor suggestion to make the API a little bit more symmetric.
>> I feel it would make more sense to move the initializer and serde to
>>
>> the
>>
>> final aggregate statement, since the serde only applies to the state
>> store,
>> and the initializer doesn't bear any relation to the first group in
>> particular. It would end up looking like this:
>>
>> KTable<K, CG> cogrouped =
>>     grouped1.cogroup(aggregator1)
>>             .cogroup(grouped2, aggregator2)
>>             .cogroup(grouped3, aggregator3)
>>             .aggregate(initializer1, aggValueSerde, storeName1);
>>
>> Alternatively, we could move the the first cogroup() method to
>> KStreamBuilder, similar to how we have .merge()
>> and end up with an api that would be even more symmetric.
>>
>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>               .cogroup(grouped2, aggregator2)
>>               .cogroup(grouped3, aggregator3)
>>               .aggregate(initializer1, aggValueSerde, storeName1);
>>
>> This doesn't have to be a blocker, but I thought it would make the API
>> just
>> a tad cleaner.
>>
>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>>
>> wrote:
>>
>> Kyle,
>>
>> Thanks a lot for the updated KIP. It looks good to me.
>>
>>
>> Guozhang
>>
>>
>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <ji...@jagunet.com>
>>
>> wrote:
>>
>> This makes much more sense to me. +1
>>
>>
>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>
>> winkelman.kyle@gmail.com>
>>
>> wrote:
>>
>> I have updated the KIP and my PR. Let me know what you think.
>> To created a cogrouped stream just call cogroup on a
>>
>> KgroupedStream
>>
>> and
>>
>> supply the initializer, aggValueSerde, and an aggregator. Then
>>
>> continue
>>
>> adding kgroupedstreams and aggregators. Then call one of the
>>
>> many
>>
>> aggregate
>>
>> calls to create a KTable.
>>
>> Thanks,
>> Kyle
>>
>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <da...@gmail.com>
>>
>> wrote:
>>
>> Hi Kyle,
>>
>> Thanks for the update. I think just one initializer makes sense
>>
>> as
>>
>> it
>>
>> should only be called once per key and generally it is just
>>
>> going
>>
>> to
>>
>> create
>>
>> a new instance of whatever the Aggregate class is.
>>
>> Cheers,
>> Damian
>>
>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>
>> winkelman.kyle@gmail.com
>>
>> wrote:
>>
>>
>> Hello all,
>>
>> I have spent some more time on this and the best alternative I
>>
>> have
>>
>> come
>>
>> up
>>
>> with is:
>> KGroupedStream has a single cogroup call that takes an
>>
>> initializer
>>
>> and
>>
>> an
>>
>> aggregator.
>> CogroupedKStream has a cogroup call that takes additional
>>
>> groupedStream
>>
>> aggregator pairs.
>> CogroupedKStream has multiple aggregate methods that create
>>
>> the
>>
>> different
>>
>> stores.
>>
>> I plan on updating the kip but I want people's input on if we
>>
>> should
>>
>> have
>>
>> the initializer be passed in once at the beginning or if we
>>
>> should
>>
>> instead
>>
>> have the initializer be required for each call to one of the
>>
>> aggregate
>>
>> calls. The first makes more sense to me but doesnt allow the
>>
>> user
>>
>> to
>>
>> specify different initializers for different tables.
>>
>> Thanks,
>> Kyle
>>
>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>
>> winkelman.kyle@gmail.com>
>>
>> wrote:
>>
>>
>> Yea I really like that idea I'll see what I can do to update
>>
>> the
>>
>> kip
>>
>> and
>>
>> my pr when I have some time. I'm not sure how well creating
>>
>> the
>>
>> kstreamaggregates will go though because at that point I will
>>
>> have
>>
>> thrown
>>
>> away the type of the values. It will be type safe I just may
>>
>> need to
>>
>> do a
>>
>> little forcing.
>>
>> Thanks,
>> Kyle
>>
>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>>
>> wrote:
>>
>> Kyle,
>>
>> Thanks for the explanations, my previous read on the wiki
>>
>> examples
>>
>> was
>>
>> wrong.
>>
>> So I guess my motivation should be "reduced" to: can we move
>>
>> the
>>
>> window
>>
>> specs param from "KGroupedStream#cogroup(..)" to
>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>
>> 1. minor: we can reduce the #.generics in CogroupedKStream
>>
>> from
>>
>> 3
>>
>> to
>>
>> 2.
>>
>> 2. major: this is for extensibility of the APIs, and since
>>
>> we
>>
>> are
>>
>> removing
>>
>> the "Evolving" annotations on Streams it may be harder to
>>
>> change it
>>
>> again
>>
>> in the future. The extended use cases are that people wanted
>>
>> to
>>
>> have
>>
>> windowed running aggregates on different granularities, e.g.
>>
>> "give
>>
>> me
>>
>> the
>>
>> counts per-minute, per-hour, per-day and per-week", and
>>
>> today
>>
>> in
>>
>> DSL
>>
>> we
>>
>> need to specify that case in multiple aggregate operators,
>>
>> which
>>
>> gets
>>
>> a
>>
>> state store / changelog, etc. And it is possible to optimize
>>
>> it
>>
>> as
>>
>> well
>>
>> to
>>
>> a single state store. Its implementation would be tricky as
>>
>> you
>>
>> need
>>
>> to
>>
>> contain different lengthed windows within your window store
>>
>> but
>>
>> just
>>
>> from
>>
>> the public API point of view, it could be specified as:
>>
>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>> "state-store-name");
>>
>> table1 = stream.aggregate(/*per-minute window*/)
>> table2 = stream.aggregate(/*per-hour window*/)
>> table3 = stream.aggregate(/*per-day window*/)
>>
>> while underlying we are only using a single store
>>
>> "state-store-name"
>>
>> for
>>
>> it.
>>
>>
>> Although this feature is out of the scope of this KIP, I'd
>>
>> like
>>
>> to
>>
>> discuss
>>
>> if we can "leave the door open" to make such changes without
>>
>> modifying
>>
>> the
>>
>> public APIs .
>>
>> Guozhang
>>
>>
>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>
>> winkelman.kyle@gmail.com
>>
>> wrote:
>>
>>
>> I allow defining a single window/sessionwindow one time
>>
>> when
>>
>> you
>>
>> make
>>
>> the
>>
>> cogroup call from a KGroupedStream. From then on you are
>>
>> using
>>
>> the
>>
>> cogroup
>>
>> call from with in CogroupedKStream which doesnt accept any
>>
>> additional
>>
>> windows/sessionwindows.
>>
>> Is this what you meant by your question or did I
>>
>> misunderstand?
>>
>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>>
>> wangguoz@gmail.com
>>
>> wrote:
>>
>> Another question that came to me is on "window alignment":
>>
>> from
>>
>> the
>>
>> KIP
>>
>> it
>>
>> seems you are allowing users to specify a (potentially
>>
>> different)
>>
>> window
>>
>> spec in each co-grouped input stream. So if these window
>>
>> specs
>>
>> are
>>
>> different how should we "align" them with different input
>>
>> streams? I
>>
>> think
>>
>> it is more natural to only specify on window spec in the
>>
>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>
>>
>> And remove it from the cogroup() functions. WDYT?
>>
>>
>> Guozhang
>>
>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>
>> wangguoz@gmail.com>
>>
>> wrote:
>>
>> Thanks for the proposal Kyle, this is a quite common use
>>
>> case
>>
>> to
>>
>> support
>>
>> such multi-way table join (i.e. N source tables with N
>>
>> aggregate
>>
>> func)
>>
>> with
>>
>> a single store and N+1 serdes, I have seen lots of people
>>
>> using
>>
>> the
>>
>> low-level PAPI to achieve this goal.
>>
>>
>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>
>> winkelman.kyle@gmail.com
>>
>> wrote:
>>
>> I like your point about not handling other cases such as
>>
>> count
>>
>> and
>>
>> reduce.
>>
>> I think that reduce may not make sense because reduce
>>
>> assumes
>>
>> that
>>
>> the
>>
>> input values are the same as the output values. With
>>
>> cogroup
>>
>> there
>>
>> may
>>
>> be
>>
>> multiple different input types and then your output type
>>
>> cant be
>>
>> multiple
>>
>> different things. In the case where you have all matching
>>
>> value
>>
>> types
>>
>> you
>>
>> can do KStreamBuilder#merge followed by the reduce.
>>
>> As for count I think it is possible to call count on all
>>
>> the
>>
>> individual
>>
>> grouped streams and then do joins. Otherwise we could
>>
>> maybe
>>
>> make
>>
>> a
>>
>> special
>>
>> call in groupedstream for this case. Because in this case
>>
>> we
>>
>> dont
>>
>> need
>>
>> to
>>
>> do type checking on the values. It could be similar to
>>
>> the
>>
>> current
>>
>> count
>>
>> methods but accept a var args of additonal grouped
>>
>> streams
>>
>> as
>>
>> well
>>
>> and
>>
>> make
>> sure they have a key type of K.
>>
>> The way I have put the kip together is to ensure that we
>>
>> do
>>
>> type
>>
>> checking.
>>
>> I don't see a way we could group them all first and then
>>
>> make a
>>
>> call
>>
>> to
>>
>> count, reduce, or aggregate because with aggregate they
>>
>> would
>>
>> need
>>
>> to
>>
>> pass
>>
>> a list of aggregators and we would have no way of type
>>
>> checking
>>
>> that
>>
>> they
>>
>> match the grouped streams.
>>
>> Thanks,
>> Kyle
>>
>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
>>
>> xavier@confluent.io>
>>
>> wrote:
>>
>> Sorry to jump on this thread so late. I agree this is a
>>
>> very
>>
>> useful
>>
>> addition and wanted to provide an additional use-case
>>
>> and
>>
>> some
>>
>> more
>>
>> comments.
>>
>> This is actually a very common analytics use-case in the
>>
>> ad-tech
>>
>> industry.
>>
>> The typical setup will have an auction stream, an
>>
>> impression
>>
>> stream,
>>
>> and a
>>
>> click stream. Those three streams need to be combined to
>>
>> compute
>>
>> aggregate
>>
>> statistics (e.g. impression statistics, and
>>
>> click-through
>>
>> rates),
>>
>> since
>>
>> most of the attributes of interest are only present the
>>
>> auction
>>
>> stream.
>>
>> A simple way to do this is to co-group all the streams
>>
>> by
>>
>> the
>>
>> auction
>>
>> key,
>>
>> and process updates to the co-group as events for each
>>
>> stream
>>
>> come
>>
>> in,
>>
>> keeping only one value from each stream before sending
>>
>> downstream
>>
>> for
>>
>> further processing / aggregation.
>>
>> One could view the result of that co-group operation as
>>
>> a
>>
>> "KTable"
>>
>> with
>>
>> multiple values per key. The key being the grouping key,
>>
>> and
>>
>> the
>>
>> values
>>
>> consisting of one value per stream.
>>
>> What I like about Kyle's approach is that allows elegant
>>
>> co-grouping
>>
>> of
>>
>> multiple streams without having to worry about the
>>
>> number
>>
>> of
>>
>> streams,
>>
>> and
>>
>> avoids dealing with Tuple types or other generic
>>
>> interfaces
>>
>> that
>>
>> could
>>
>> get
>>
>> messy if we wanted to preserve all the value types in
>>
>> the
>>
>> resulting
>>
>> co-grouped stream.
>>
>> My only concern is that we only allow the cogroup +
>>
>> aggregate
>>
>> combined
>>
>> operation. This forces the user to build their own tuple
>>
>> serialization
>>
>> format if they want to preserve the individual input
>>
>> stream
>>
>> values
>>
>> as
>>
>> a
>>
>> group. It also deviates quite a bit from our approach in
>>
>> KGroupedStream
>>
>> which offers other operations, such as count and reduce,
>>
>> which
>>
>> should
>>
>> also
>>
>> be applicable to a co-grouped stream.
>>
>> Overall I still think this is a really useful addition,
>>
>> but
>>
>> I
>>
>> feel
>>
>> we
>>
>> haven't spend much time trying to explore alternative
>>
>> DSLs
>>
>> that
>>
>> could
>>
>> maybe
>>
>> generalize better or match our existing syntax more
>>
>> closely.
>>
>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>
>> winkelman.kyle@gmail.com
>>
>> wrote:
>>
>>
>> Eno, is there anyone else that is an expert in the
>>
>> kafka
>>
>> streams
>>
>> realm
>>
>> that
>>
>> I should reach out to for input?
>>
>> I believe Damian Guy is still planning on reviewing
>>
>> this
>>
>> more
>>
>> in
>>
>> depth
>>
>> so I
>>
>>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
I'd agree that the aggregate value serde and the initializer does not bear
direct relationship with the first `cogroup` calls, but after I tried to
write some example code with these two different set of APIs I felt the
current APIs just program more naturally.

I know it is kinda subjective, but I do think that user experience may be
more important as a deciding factor than the logical argument for public
interfaces. So I'd recommend people to also try out writing some example
lines also and we can circle back and discuss which one feels more natural
to write code.


Guozhang


On Fri, Jun 9, 2017 at 1:59 AM, Michal Borowiecki <
michal.borowiecki@openbet.com> wrote:

> I feel it would make more sense to move the initializer and serde to the
> final aggregate statement, since the serde only applies to the state store,
> and the initializer doesn't bear any relation to the first group in
> particular.
>
> +1 for moving initializer and serde from cogroup() to the aggregate() for
> the reasons mentioned above.
>
> Cheers,
>
> Michał
>
> On 08/06/17 22:44, Guozhang Wang wrote:
>
> Note that although the internal `AbstractStoreSupplier` does maintain the
> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
> to always retain that information, and hence we cannot assume that
> StateStoreSuppliers always retain key / value serdes.
>
> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io> wrote:
>
>
> Another reason for the serde not to be in the first cogroup call, is that
> the serde should not be required if you pass a StateStoreSupplier to
> aggregate()
>
> Regarding the aggregated type <T> I don't the why initializer should be
> favored over aggregator to define the type. In my mind separating the
> initializer into the last aggregate call clearly indicates that the
> initializer is independent of any of the aggregators or streams and that we
> don't wait for grouped1 events to initialize the co-group.
>
> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com> wrote:
>
>
> On a second thought... This is the current proposal API
>
>
> ```
>
> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>
> final
>
> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> aggValueSerde)
>
> ```
>
>
> If we do not have the initializer in the first co-group it might be a bit
> awkward for users to specify the aggregator that returns a typed <T>
>
> value?
>
> Maybe it is still better to put these two functions in the same api?
>
>
>
> Guozhang
>
> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>
> wrote:
>
> This suggestion lgtm. I would vote for the first alternative than
>
> adding
>
> it to the `KStreamBuilder` though.
>
> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> <xa...@confluent.io>
> wrote:
>
>
> I have a minor suggestion to make the API a little bit more symmetric.
> I feel it would make more sense to move the initializer and serde to
>
> the
>
> final aggregate statement, since the serde only applies to the state
> store,
> and the initializer doesn't bear any relation to the first group in
> particular. It would end up looking like this:
>
> KTable<K, CG> cogrouped =
>     grouped1.cogroup(aggregator1)
>             .cogroup(grouped2, aggregator2)
>             .cogroup(grouped3, aggregator3)
>             .aggregate(initializer1, aggValueSerde, storeName1);
>
> Alternatively, we could move the the first cogroup() method to
> KStreamBuilder, similar to how we have .merge()
> and end up with an api that would be even more symmetric.
>
> KStreamBuilder.cogroup(grouped1, aggregator1)
>               .cogroup(grouped2, aggregator2)
>               .cogroup(grouped3, aggregator3)
>               .aggregate(initializer1, aggValueSerde, storeName1);
>
> This doesn't have to be a blocker, but I thought it would make the API
> just
> a tad cleaner.
>
> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> <wa...@gmail.com>
>
> wrote:
>
> Kyle,
>
> Thanks a lot for the updated KIP. It looks good to me.
>
>
> Guozhang
>
>
> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> <ji...@jagunet.com>
>
> wrote:
>
> This makes much more sense to me. +1
>
>
> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>
> winkelman.kyle@gmail.com>
>
> wrote:
>
> I have updated the KIP and my PR. Let me know what you think.
> To created a cogrouped stream just call cogroup on a
>
> KgroupedStream
>
> and
>
> supply the initializer, aggValueSerde, and an aggregator. Then
>
> continue
>
> adding kgroupedstreams and aggregators. Then call one of the
>
> many
>
> aggregate
>
> calls to create a KTable.
>
> Thanks,
> Kyle
>
> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> <da...@gmail.com>
>
> wrote:
>
> Hi Kyle,
>
> Thanks for the update. I think just one initializer makes sense
>
> as
>
> it
>
> should only be called once per key and generally it is just
>
> going
>
> to
>
> create
>
> a new instance of whatever the Aggregate class is.
>
> Cheers,
> Damian
>
> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
>
> Hello all,
>
> I have spent some more time on this and the best alternative I
>
> have
>
> come
>
> up
>
> with is:
> KGroupedStream has a single cogroup call that takes an
>
> initializer
>
> and
>
> an
>
> aggregator.
> CogroupedKStream has a cogroup call that takes additional
>
> groupedStream
>
> aggregator pairs.
> CogroupedKStream has multiple aggregate methods that create
>
> the
>
> different
>
> stores.
>
> I plan on updating the kip but I want people's input on if we
>
> should
>
> have
>
> the initializer be passed in once at the beginning or if we
>
> should
>
> instead
>
> have the initializer be required for each call to one of the
>
> aggregate
>
> calls. The first makes more sense to me but doesnt allow the
>
> user
>
> to
>
> specify different initializers for different tables.
>
> Thanks,
> Kyle
>
> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>
> winkelman.kyle@gmail.com>
>
> wrote:
>
>
> Yea I really like that idea I'll see what I can do to update
>
> the
>
> kip
>
> and
>
> my pr when I have some time. I'm not sure how well creating
>
> the
>
> kstreamaggregates will go though because at that point I will
>
> have
>
> thrown
>
> away the type of the values. It will be type safe I just may
>
> need to
>
> do a
>
> little forcing.
>
> Thanks,
> Kyle
>
> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>
> wrote:
>
> Kyle,
>
> Thanks for the explanations, my previous read on the wiki
>
> examples
>
> was
>
> wrong.
>
> So I guess my motivation should be "reduced" to: can we move
>
> the
>
> window
>
> specs param from "KGroupedStream#cogroup(..)" to
> "CogroupedKStream#aggregate(..)", and my motivations are:
>
> 1. minor: we can reduce the #.generics in CogroupedKStream
>
> from
>
> 3
>
> to
>
> 2.
>
> 2. major: this is for extensibility of the APIs, and since
>
> we
>
> are
>
> removing
>
> the "Evolving" annotations on Streams it may be harder to
>
> change it
>
> again
>
> in the future. The extended use cases are that people wanted
>
> to
>
> have
>
> windowed running aggregates on different granularities, e.g.
>
> "give
>
> me
>
> the
>
> counts per-minute, per-hour, per-day and per-week", and
>
> today
>
> in
>
> DSL
>
> we
>
> need to specify that case in multiple aggregate operators,
>
> which
>
> gets
>
> a
>
> state store / changelog, etc. And it is possible to optimize
>
> it
>
> as
>
> well
>
> to
>
> a single state store. Its implementation would be tricky as
>
> you
>
> need
>
> to
>
> contain different lengthed windows within your window store
>
> but
>
> just
>
> from
>
> the public API point of view, it could be specified as:
>
> CogroupedKStream stream = stream1.cogroup(stream2, ...
> "state-store-name");
>
> table1 = stream.aggregate(/*per-minute window*/)
> table2 = stream.aggregate(/*per-hour window*/)
> table3 = stream.aggregate(/*per-day window*/)
>
> while underlying we are only using a single store
>
> "state-store-name"
>
> for
>
> it.
>
>
> Although this feature is out of the scope of this KIP, I'd
>
> like
>
> to
>
> discuss
>
> if we can "leave the door open" to make such changes without
>
> modifying
>
> the
>
> public APIs .
>
> Guozhang
>
>
> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
>
> I allow defining a single window/sessionwindow one time
>
> when
>
> you
>
> make
>
> the
>
> cogroup call from a KGroupedStream. From then on you are
>
> using
>
> the
>
> cogroup
>
> call from with in CogroupedKStream which doesnt accept any
>
> additional
>
> windows/sessionwindows.
>
> Is this what you meant by your question or did I
>
> misunderstand?
>
> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>
> wangguoz@gmail.com
>
> wrote:
>
> Another question that came to me is on "window alignment":
>
> from
>
> the
>
> KIP
>
> it
>
> seems you are allowing users to specify a (potentially
>
> different)
>
> window
>
> spec in each co-grouped input stream. So if these window
>
> specs
>
> are
>
> different how should we "align" them with different input
>
> streams? I
>
> think
>
> it is more natural to only specify on window spec in the
>
> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>
>
> And remove it from the cogroup() functions. WDYT?
>
>
> Guozhang
>
> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>
> wangguoz@gmail.com>
>
> wrote:
>
> Thanks for the proposal Kyle, this is a quite common use
>
> case
>
> to
>
> support
>
> such multi-way table join (i.e. N source tables with N
>
> aggregate
>
> func)
>
> with
>
> a single store and N+1 serdes, I have seen lots of people
>
> using
>
> the
>
> low-level PAPI to achieve this goal.
>
>
> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
> I like your point about not handling other cases such as
>
> count
>
> and
>
> reduce.
>
> I think that reduce may not make sense because reduce
>
> assumes
>
> that
>
> the
>
> input values are the same as the output values. With
>
> cogroup
>
> there
>
> may
>
> be
>
> multiple different input types and then your output type
>
> cant be
>
> multiple
>
> different things. In the case where you have all matching
>
> value
>
> types
>
> you
>
> can do KStreamBuilder#merge followed by the reduce.
>
> As for count I think it is possible to call count on all
>
> the
>
> individual
>
> grouped streams and then do joins. Otherwise we could
>
> maybe
>
> make
>
> a
>
> special
>
> call in groupedstream for this case. Because in this case
>
> we
>
> dont
>
> need
>
> to
>
> do type checking on the values. It could be similar to
>
> the
>
> current
>
> count
>
> methods but accept a var args of additonal grouped
>
> streams
>
> as
>
> well
>
> and
>
> make
> sure they have a key type of K.
>
> The way I have put the kip together is to ensure that we
>
> do
>
> type
>
> checking.
>
> I don't see a way we could group them all first and then
>
> make a
>
> call
>
> to
>
> count, reduce, or aggregate because with aggregate they
>
> would
>
> need
>
> to
>
> pass
>
> a list of aggregators and we would have no way of type
>
> checking
>
> that
>
> they
>
> match the grouped streams.
>
> Thanks,
> Kyle
>
> On May 19, 2017 11:42 AM, "Xavier Léauté" <
>
> xavier@confluent.io>
>
> wrote:
>
> Sorry to jump on this thread so late. I agree this is a
>
> very
>
> useful
>
> addition and wanted to provide an additional use-case
>
> and
>
> some
>
> more
>
> comments.
>
> This is actually a very common analytics use-case in the
>
> ad-tech
>
> industry.
>
> The typical setup will have an auction stream, an
>
> impression
>
> stream,
>
> and a
>
> click stream. Those three streams need to be combined to
>
> compute
>
> aggregate
>
> statistics (e.g. impression statistics, and
>
> click-through
>
> rates),
>
> since
>
> most of the attributes of interest are only present the
>
> auction
>
> stream.
>
> A simple way to do this is to co-group all the streams
>
> by
>
> the
>
> auction
>
> key,
>
> and process updates to the co-group as events for each
>
> stream
>
> come
>
> in,
>
> keeping only one value from each stream before sending
>
> downstream
>
> for
>
> further processing / aggregation.
>
> One could view the result of that co-group operation as
>
> a
>
> "KTable"
>
> with
>
> multiple values per key. The key being the grouping key,
>
> and
>
> the
>
> values
>
> consisting of one value per stream.
>
> What I like about Kyle's approach is that allows elegant
>
> co-grouping
>
> of
>
> multiple streams without having to worry about the
>
> number
>
> of
>
> streams,
>
> and
>
> avoids dealing with Tuple types or other generic
>
> interfaces
>
> that
>
> could
>
> get
>
> messy if we wanted to preserve all the value types in
>
> the
>
> resulting
>
> co-grouped stream.
>
> My only concern is that we only allow the cogroup +
>
> aggregate
>
> combined
>
> operation. This forces the user to build their own tuple
>
> serialization
>
> format if they want to preserve the individual input
>
> stream
>
> values
>
> as
>
> a
>
> group. It also deviates quite a bit from our approach in
>
> KGroupedStream
>
> which offers other operations, such as count and reduce,
>
> which
>
> should
>
> also
>
> be applicable to a co-grouped stream.
>
> Overall I still think this is a really useful addition,
>
> but
>
> I
>
> feel
>
> we
>
> haven't spend much time trying to explore alternative
>
> DSLs
>
> that
>
> could
>
> maybe
>
> generalize better or match our existing syntax more
>
> closely.
>
> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
>
> Eno, is there anyone else that is an expert in the
>
> kafka
>
> streams
>
> realm
>
> that
>
> I should reach out to for input?
>
> I believe Damian Guy is still planning on reviewing
>
> this
>
> more
>
> in
>
> depth
>
> so I
>
> will wait for his inputs before continuing.
>
> On May 9, 2017 7:30 AM, "Eno Thereska" <
>
> eno.thereska@gmail.com
>
> wrote:
>
> Thanks Kyle, good arguments.
>
> Eno
>
>
> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
> *- minor: could you add an exact example (similar to
>
> what
>
> Jay’s
>
> example
>
> is,
>
> or like your Spark/Pig pointers had) to make this
>
> super
>
> concrete?*
>
> I have added a more concrete example to the KIP.
>
> *- my main concern is that we’re exposing this
>
> optimization
>
> to
>
> the
>
> DSL.
>
> In
>
> an ideal world, an optimizer would take the existing
>
> DSL
>
> and
>
> do
>
> the
>
> right
>
> thing under the covers (create just one state store,
>
> arrange
>
> the
>
> nodes
>
> etc). The original DSL had a bunch of small,
>
> composable
>
> pieces
>
> (group,
>
> aggregate, join) that this proposal groups together.
>
> I’d
>
> like to
>
> hear
>
> your
>
> thoughts on whether it’s possible to do this
>
> optimization
>
> with
>
> the
>
> current
>
> DSL, at the topology builder level.*
> You would have to make a lot of checks to understand
>
> if
>
> it
>
> is
>
> even
>
> possible
>
> to make this optimization:
> 1. Make sure they are all KTableKTableOuterJoins
> 2. None of the intermediate KTables are used for
>
> anything
>
> else.
>
> 3. None of the intermediate stores are used. (This
>
> may
>
> be
>
> impossible
>
> especially if they use KafkaStreams#store after the
>
> topology
>
> has
>
> already
>
> been built.)
> You would then need to make decisions during the
>
> optimization:
>
> 1. Your new initializer would the composite of all
>
> the
>
> individual
>
> initializers and the valueJoiners.
> 2. I am having a hard time thinking about how you
>
> would
>
> turn
>
> the
>
> aggregators and valueJoiners into an aggregator that
>
> would
>
> work
>
> on
>
> the
>
> final object, but this may be possible.
> 3. Which state store would you use? The ones declared
>
> would
>
> be
>
> for
>
> the
>
> aggregate values. None of the declared ones would be
>
> guaranteed
>
> to
>
> hold
>
> the
>
> final object. This would mean you must created a new
>
> state
>
> store
>
> and
>
> not
>
> created any of the declared ones.
>
> The main argument I have against it is even if it
>
> could
>
> be
>
> done
>
> I
>
> don't
>
> know that we would want to have this be an
>
> optimization
>
> in
>
> the
>
> background
>
> because the user would still be required to think
>
> about
>
> all
>
> of
>
> the
>
> intermediate values that they shouldn't need to worry
>
> about
>
> if
>
> they
>
> only
>
> care about the final object.
>
> In my opinion cogroup is a common enough case that it
>
> should
>
> be
>
> part
>
> of
>
> the
>
> composable pieces (group, aggregate, join) because we
>
> want
>
> to
>
> allow
>
> people
>
> to join more than 2 or more streams in an easy way.
>
> Right
>
> now I
>
> don't
>
> think
>
> we give them ways of handling this use case easily.
>
> *-I think there will be scope for several such
>
> optimizations
>
> in
>
> the
>
> future
>
> and perhaps at some point we need to think about
>
> decoupling
>
> the
>
> 1:1
>
> mapping
>
> from the DSL into the physical topology.*
> I would argue that cogroup is not just an
>
> optimization
>
> it
>
> is
>
> a
>
> new
>
> way
>
> for
>
> the users to look at accomplishing a problem that
>
> requires
>
> multiple
>
> streams. I may sound like a broken record but I don't
>
> think
>
> users
>
> should
>
> have to build the N-1 intermediate tables and deal
>
> with
>
> their
>
> initializers,
>
> serdes and stores if all they care about is the final
>
> object.
>
> Now if for example someone uses cogroup but doesn't
>
> supply
>
> additional
>
> streams and aggregators this case is equivalent to a
>
> single
>
> grouped
>
> stream
>
> making an aggregate call. This case is what I view an
>
> optimization
>
> as,
>
> we
>
> could remove the KStreamCogroup and act as if there
>
> was
>
> just
>
> a
>
> call
>
> to
>
> KGroupedStream#aggregate instead of calling
>
> KGroupedStream#cogroup.
>
> (I
>
> would prefer to just write a warning saying that this
>
> is
>
> not
>
> how
>
> cogroup
>
> is
>
> to be used.)
>
> Thanks,
> Kyle
>
> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
>
> eno.thereska@gmail.com
>
> wrote:
>
> Hi Kyle,
>
> Thanks for the KIP again. A couple of comments:
>
> - minor: could you add an exact example (similar to
>
> what
>
> Jay’s
>
> example
>
> is,
>
> or like your Spark/Pig pointers had) to make this
>
> super
>
> concrete?
>
> - my main concern is that we’re exposing this
>
> optimization
>
> to
>
> the
>
> DSL.
>
> In
>
> an ideal world, an optimizer would take the existing
>
> DSL
>
> and do
>
> the
>
> right
>
> thing under the covers (create just one state store,
>
> arrange
>
> the
>
> nodes
>
> etc). The original DSL had a bunch of small,
>
> composable
>
> pieces
>
> (group,
>
> aggregate, join) that this proposal groups together.
>
> I’d
>
> like
>
> to
>
> hear
>
> your
>
> thoughts on whether it’s possible to do this
>
> optimization
>
> with
>
> the
>
> current
>
> DSL, at the topology builder level.
>
> I think there will be scope for several such
>
> optimizations
>
> in
>
> the
>
> future
>
> and perhaps at some point we need to think about
>
> decoupling
>
> the
>
> 1:1
>
> mapping
>
> from the DSL into the physical topology.
>
> Thanks
> Eno
>
>
> On May 5, 2017, at 4:39 PM, Jay Kreps <
>
> jay@confluent.io>
>
> wrote:
>
> I haven't digested the proposal but the use case is
>
> pretty
>
> common.
>
> An
>
> example would be the "customer 360" or "unified
>
> customer
>
> profile"
>
> use
>
> case
>
> we often use. In that use case you have a dozen
>
> systems
>
> each
>
> of
>
> which
>
> has
>
> some information about your customer (account
>
> details,
>
> settings,
>
> billing
>
> info, customer service contacts, purchase history,
>
> etc).
>
> Your
>
> goal
>
> is
>
> to
>
> join/munge these into a single profile record for
>
> each
>
> customer
>
> that
>
> has
>
> all the relevant info in a usable form and is
>
> up-to-date
>
> with
>
> all
>
> the
>
> source systems. If you implement that with kstreams
>
> as
>
> a
>
> sequence
>
> of
>
> joins
>
> i think today we'd fully materialize N-1
>
> intermediate
>
> tables.
>
> But
>
> clearly
>
> you only need a single stage to group all these
>
> things
>
> that
>
> are
>
> already
>
> co-partitioned. A distributed database would do
>
> this
>
> under
>
> the
>
> covers
>
> which
>
> is arguably better (at least when it does the right
>
> thing)
>
> and
>
> perhaps
>
> we
>
> could do the same thing but I'm not sure we know
>
> the
>
> partitioning
>
> so
>
> we
>
> may
>
> need an explicit cogroup command that impllies they
>
> are
>
> already
>
> co-partitioned.
>
> -Jay
>
> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
>
> Yea thats a good way to look at it.
> I have seen this type of functionality in a couple
>
> other
>
> platforms
>
> like
>
> spark and pig.https://spark.apache.org/docs/
>
> 0.6.2/api/core/spark/
>
> PairRDDFunctions.html
>
> https://www.tutorialspoint.com/apache_pig/apache_pig_
>
> cogroup_operator.htm
>
>
> On May 5, 2017 7:43 AM, "Damian Guy" <
>
> damian.guy@gmail.com>
>
> wrote:
>
> Hi Kyle,
>
> If i'm reading this correctly it is like an N way
>
> outer
>
> join?
>
> So
>
> an
>
> input
>
> on any stream will always produce a new
>
> aggregated
>
> value
>
> -
>
> is
>
> that
>
> correct?
>
> Effectively, each Aggregator just looks up the
>
> current
>
> value,
>
> aggregates
>
> and forwards the result.
> I need to look into it and think about it a bit
>
> more,
>
> but it
>
> seems
>
> like
>
> it
>
> could be a useful optimization.
>
> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
>
> I sure can. I have added the following
>
> description
>
> to
>
> my
>
> KIP. If
>
> this
>
> doesn't help let me know and I will take some
>
> more
>
> time
>
> to
>
> build a
>
> diagram
>
> and make more of a step by step description:
>
> Example with Current API:
>
> KTable<K, V1> table1 =
> builder.stream("topic1").group
>
> ByKey().aggregate(
>
> initializer1
>
> ,
>
> aggregator1,
>
> aggValueSerde1, storeName1);
> KTable<K, V2> table2 =
> builder.stream("topic2").group
>
> ByKey().aggregate(
>
> initializer2
>
> ,
>
> aggregator2,
>
> aggValueSerde2, storeName2);
> KTable<K, V3> table3 =
> builder.stream("topic3").group
>
> ByKey().aggregate(
>
> initializer3
>
> ,
>
> aggregator3,
>
> aggValueSerde3, storeName3);
> KTable<K, CG> cogrouped =
>
> table1.outerJoin(table2,
>
> joinerOneAndTwo).outerJoin(table3,
>
> joinerOneTwoAndThree);
>
> As you can see this creates 3 StateStores,
>
> requires
>
> 3
>
> initializers,
>
> and 3
>
> aggValueSerdes. This also adds the pressure to
>
> user
>
> to
>
> define
>
> what
>
> the
>
> intermediate values are going to be (V1, V2,
>
> V3).
>
> They
>
> are
>
> left
>
> with a
>
> couple choices, first to make V1, V2, and V3 all
>
> the
>
> same
>
> as
>
> CG
>
> and
>
> the
>
> two
>
> joiners are more like mergers, or second make
>
> them
>
> intermediate
>
> states
>
> such
>
> as Topic1Map, Topic2Map, and Topic3Map and the
>
> joiners
>
> use
>
> those
>
> to
>
> build
>
> the final aggregate CG value. This is something
>
> the
>
> user
>
> could
>
> avoid
>
> thinking about with this KIP.
>
> When a new input arrives lets say at "topic1" it
>
> will
>
> first
>
> go
>
> through
>
> a
>
> KStreamAggregate grabbing the current aggregate
>
> from
>
> storeName1.
>
> It
>
> will
>
> produce this in the form of the first
>
> intermediate
>
> value
>
> and
>
> get
>
> sent
>
> through a KTableKTableOuterJoin where it will
>
> look
>
> up
>
> the
>
> current
>
> value
>
> of
>
> the key in storeName2. It will use the first
>
> joiner
>
> to
>
> calculate
>
> the
>
> second
>
> intermediate value, which will go through an
>
> additional
>
> KTableKTableOuterJoin. Here it will look up the
>
> current
>
> value of
>
> the
>
> key
>
> in
>
> storeName3 and use the second joiner to build
>
> the
>
> final
>
> aggregate
>
> value.
>
> If you think through all possibilities for
>
> incoming
>
> topics
>
> you
>
> will
>
> see
>
> that no matter which topic it comes in through
>
> all
>
> three
>
> stores
>
> are
>
> queried
>
> and all of the joiners must get used.
>
> Topology wise for N incoming streams this
>
> creates
>
> N
>
> KStreamAggregates, 2*(N-1)
>
> KTableKTableOuterJoins,
>
> and
>
> N-1
>
> KTableKTableJoinMergers.
>
>
>
> Example with Proposed API:
>
> KGroupedStream<K, V1> grouped1 =
>
> builder.stream("topic1").
>
> groupByKey();
>
> KGroupedStream<K, V2> grouped2 =
>
> builder.stream("topic2").
>
> groupByKey();
>
> KGroupedStream<K, V3> grouped3 =
>
> builder.stream("topic3").
>
> groupByKey();
>
> KTable<K, CG> cogrouped =
>
> grouped1.cogroup(initializer1,
>
> aggregator1,
>
> aggValueSerde1, storeName1)
>      .cogroup(grouped2, aggregator2)
>      .cogroup(grouped3, aggregator3)
>      .aggregate();
>
> As you can see this creates 1 StateStore,
>
> requires 1
>
> initializer,
>
> and
>
> 1
>
> aggValueSerde. The user no longer has to worry
>
> about
>
> the
>
> intermediate
>
> values and the joiners. All they have to think
>
> about
>
> is
>
> how
>
> each
>
> stream
>
> impacts the creation of the final CG object.
>
> When a new input arrives lets say at "topic1" it
>
> will
>
> first
>
> go
>
> through
>
> a
>
> KStreamAggreagte and grab the current aggregate
>
> from
>
> storeName1.
>
> It
>
> will
>
> add its incoming object to the aggregate, update
>
> the
>
> store
>
> and
>
> pass
>
> the
>
> new
>
> aggregate on. This new aggregate goes through
>
> the
>
> KStreamCogroup
>
> which
>
> is
>
> pretty much just a pass through processor and
>
> you
>
> are
>
> done.
>
> Topology wise for N incoming streams the new api
>
> will
>
> only
>
> every
>
> create N
>
> KStreamAggregates and 1 KStreamCogroup.
>
> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax
>
> <
>
> matthias@confluent.io
>
> wrote:
>
>
> Kyle,
>
> thanks a lot for the KIP. Maybe I am a little
>
> slow,
>
> but I
>
> could
>
> not
>
> follow completely. Could you maybe add a more
>
> concrete
>
> example,
>
> like
>
> 3
>
> streams with 3 records each (plus expected
>
> result),
>
> and
>
> show
>
> the
>
> difference between current way to to implement
>
> it
>
> and
>
> the
>
> proposed
>
> API?
>
> This could also cover the internal processing
>
> to
>
> see
>
> what
>
> store
>
> calls
>
> would be required for both approaches etc.
>
> I think, it's pretty advanced stuff you
>
> propose,
>
> and
>
> it
>
> would
>
> help
>
> to
>
> understand it better.
>
> Thanks a lot!
>
>
> -Matthias
>
>
>
> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
>
> I have made a pull request. It can be found
>
> here.
>
> https://github.com/apache/kafka/pull/2975
>
> I plan to write some more unit tests for my
>
> classes
>
> and
>
> get
>
> around
>
> to
>
> writing documentation for the public api
>
> additions.
>
> One thing I was curious about is during the
>
> KCogroupedStreamImpl#aggregate
>
> method I pass null to the KGroupedStream#
>
> repartitionIfRequired
>
> method.
>
> I
>
> can't supply the store name because if more
>
> than
>
> one
>
> grouped
>
> stream
>
> repartitions an error is thrown. Is there some
>
> name
>
> that
>
> someone
>
> can
>
> recommend or should I leave the null and allow
>
> it
>
> to
>
> fall
>
> back
>
> to
>
> the
>
> KGroupedStream.name?
>
> Should this be expanded to handle grouped
>
> tables?
>
> This
>
> would
>
> be
>
> pretty
>
> easy
>
> for a normal aggregate but one allowing
>
> session
>
> stores
>
> and
>
> windowed
>
> stores
>
> would required KTableSessionWindowAggregate
>
> and
>
> KTableWindowAggregate
>
> implementations.
>
> Thanks,
> Kyle
>
> On May 4, 2017 1:24 PM, "Eno Thereska" <
>
> eno.thereska@gmail.com>
>
> wrote:
>
> I’ll look as well asap, sorry, been swamped.
>
> Eno
>
> On May 4, 2017, at 6:17 PM, Damian Guy <
>
> damian.guy@gmail.com>
>
> wrote:
>
> Hi Kyle,
>
> Thanks for the KIP. I apologize that i
>
> haven't
>
> had
>
> the
>
> chance
>
> to
>
> look
>
> at
>
> the KIP yet, but will schedule some time to
>
> look
>
> into
>
> it
>
> tomorrow.
>
> For
>
> the
>
> implementation, can you raise a PR against
>
> kafka
>
> trunk
>
> and
>
> mark
>
> it
>
> as
>
> WIP?
>
> It will be easier to review what you have
>
> done.
>
> Thanks,
> Damian
>
> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
>
> winkelman.kyle@gmail.com
>
> wrote:
>
> I am replying to this in hopes it will draw
>
> some
>
> attention
>
> to
>
> my
>
> KIP
>
> as
>
> I
>
> haven't heard from anyone in a couple days.
>
> This
>
> is my
>
> first
>
> KIP
>
> and
>
> my
>
> first large contribution to the project so
>
> I'm
>
> sure I
>
> did
>
> something
>
> wrong.
>
> ;)
>
> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
>
> winkelman.kyle@gmail.com>
>
> wrote:
>
> Hello all,
>
> I have created KIP-150 to facilitate
>
> discussion
>
> about
>
> adding
>
> cogroup
>
> to
>
> the streams DSL.
>
> Please find the KIP here:https://cwiki.apache.org/
>
> confluence/display/KAFKA/KIP-
>
> 150+-+Kafka-Streams+Cogroup
>
> Please find my initial implementation
>
> here:
>
> https://github.com/KyleWinkelman/kafka
>
> Thanks,
> Kyle Winkelman
>
>
>
>
> --
> -- Guozhang
>
>
>
>
> --
> -- Guozhang
>
>
>
>
> --
> -- Guozhang
>
>
>
> --
> -- Guozhang
>
>
>
>
> --
> -- Guozhang
>
>
>
>
> --
> -- Guozhang
>
>
>
>
> --
> <http://www.openbet.com/> Michal Borowiecki
> Senior Software Engineer L4
> T: +44 208 742 1600 <+44%2020%208742%201600>
>
>
> +44 203 249 8448 <+44%2020%203249%208448>
>
>
>
> E: michal.borowiecki@openbet.com
> W: www.openbet.com
> OpenBet Ltd
>
> Chiswick Park Building 9
>
> 566 Chiswick High Rd
>
> London
>
> W4 5XT
>
> UK
> <https://www.openbet.com/email_promo>
> This message is confidential and intended only for the addressee. If you
> have received this message in error, please immediately notify the
> postmaster@openbet.com and delete it from your system as well as any
> copies. The content of e-mails as well as traffic data may be monitored by
> OpenBet for employment and security purposes. To protect the environment
> please do not print this e-mail unless necessary. OpenBet Ltd. Registered
> Office: Chiswick Park Building 9, 566 Chiswick High Road, London, W4 5XT,
> United Kingdom. A company registered in England and Wales. Registered no.
> 3134634. VAT no. GB927523612
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Michal Borowiecki <mi...@openbet.com>.
> I feel it would make more sense to move the initializer and serde to the
> final aggregate statement, since the serde only applies to the state 
> store,
> and the initializer doesn't bear any relation to the first group in
> particular.
+1 for moving initializer and serde from cogroup() to the aggregate() 
for the reasons mentioned above.

Cheers,

Michał


On 08/06/17 22:44, Guozhang Wang wrote:
> Note that although the internal `AbstractStoreSupplier` does maintain the
> key-value serdes, we do not enforce the interface of `StateStoreSupplier`
> to always retain that information, and hence we cannot assume that
> StateStoreSuppliers always retain key / value serdes.
>
> On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> wrote:
>
>> Another reason for the serde not to be in the first cogroup call, is that
>> the serde should not be required if you pass a StateStoreSupplier to
>> aggregate()
>>
>> Regarding the aggregated type <T> I don't the why initializer should be
>> favored over aggregator to define the type. In my mind separating the
>> initializer into the last aggregate call clearly indicates that the
>> initializer is independent of any of the aggregators or streams and that we
>> don't wait for grouped1 events to initialize the co-group.
>>
>> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> On a second thought... This is the current proposal API
>>>
>>>
>>> ```
>>>
>>> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
>> final
>>> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
>>> aggValueSerde)
>>>
>>> ```
>>>
>>>
>>> If we do not have the initializer in the first co-group it might be a bit
>>> awkward for users to specify the aggregator that returns a typed <T>
>> value?
>>> Maybe it is still better to put these two functions in the same api?
>>>
>>>
>>>
>>> Guozhang
>>>
>>> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>>>> This suggestion lgtm. I would vote for the first alternative than
>> adding
>>>> it to the `KStreamBuilder` though.
>>>>
>>>> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
>>>> wrote:
>>>>
>>>>> I have a minor suggestion to make the API a little bit more symmetric.
>>>>> I feel it would make more sense to move the initializer and serde to
>> the
>>>>> final aggregate statement, since the serde only applies to the state
>>>>> store,
>>>>> and the initializer doesn't bear any relation to the first group in
>>>>> particular. It would end up looking like this:
>>>>>
>>>>> KTable<K, CG> cogrouped =
>>>>>      grouped1.cogroup(aggregator1)
>>>>>              .cogroup(grouped2, aggregator2)
>>>>>              .cogroup(grouped3, aggregator3)
>>>>>              .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>
>>>>> Alternatively, we could move the the first cogroup() method to
>>>>> KStreamBuilder, similar to how we have .merge()
>>>>> and end up with an api that would be even more symmetric.
>>>>>
>>>>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>>>>                .cogroup(grouped2, aggregator2)
>>>>>                .cogroup(grouped3, aggregator3)
>>>>>                .aggregate(initializer1, aggValueSerde, storeName1);
>>>>>
>>>>> This doesn't have to be a blocker, but I thought it would make the API
>>>>> just
>>>>> a tad cleaner.
>>>>>
>>>>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
>>> wrote:
>>>>>> Kyle,
>>>>>>
>>>>>> Thanks a lot for the updated KIP. It looks good to me.
>>>>>>
>>>>>>
>>>>>> Guozhang
>>>>>>
>>>>>>
>>>>>> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
>>> wrote:
>>>>>>> This makes much more sense to me. +1
>>>>>>>
>>>>>>>> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>>>>> winkelman.kyle@gmail.com>
>>>>>>> wrote:
>>>>>>>> I have updated the KIP and my PR. Let me know what you think.
>>>>>>>> To created a cogrouped stream just call cogroup on a
>>> KgroupedStream
>>>>> and
>>>>>>>> supply the initializer, aggValueSerde, and an aggregator. Then
>>>>> continue
>>>>>>>> adding kgroupedstreams and aggregators. Then call one of the
>> many
>>>>>>> aggregate
>>>>>>>> calls to create a KTable.
>>>>>>>>
>>>>>>>> Thanks,
>>>>>>>> Kyle
>>>>>>>>
>>>>>>>> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com>
>>> wrote:
>>>>>>>>> Hi Kyle,
>>>>>>>>>
>>>>>>>>> Thanks for the update. I think just one initializer makes sense
>>> as
>>>>> it
>>>>>>>>> should only be called once per key and generally it is just
>> going
>>>>> to
>>>>>>> create
>>>>>>>>> a new instance of whatever the Aggregate class is.
>>>>>>>>>
>>>>>>>>> Cheers,
>>>>>>>>> Damian
>>>>>>>>>
>>>>>>>>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>>>>> winkelman.kyle@gmail.com
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hello all,
>>>>>>>>>>
>>>>>>>>>> I have spent some more time on this and the best alternative I
>>>>> have
>>>>>>> come
>>>>>>>>> up
>>>>>>>>>> with is:
>>>>>>>>>> KGroupedStream has a single cogroup call that takes an
>>> initializer
>>>>>> and
>>>>>>> an
>>>>>>>>>> aggregator.
>>>>>>>>>> CogroupedKStream has a cogroup call that takes additional
>>>>>> groupedStream
>>>>>>>>>> aggregator pairs.
>>>>>>>>>> CogroupedKStream has multiple aggregate methods that create
>> the
>>>>>>> different
>>>>>>>>>> stores.
>>>>>>>>>>
>>>>>>>>>> I plan on updating the kip but I want people's input on if we
>>>>> should
>>>>>>> have
>>>>>>>>>> the initializer be passed in once at the beginning or if we
>>> should
>>>>>>>>> instead
>>>>>>>>>> have the initializer be required for each call to one of the
>>>>>> aggregate
>>>>>>>>>> calls. The first makes more sense to me but doesnt allow the
>>> user
>>>>> to
>>>>>>>>>> specify different initializers for different tables.
>>>>>>>>>>
>>>>>>>>>> Thanks,
>>>>>>>>>> Kyle
>>>>>>>>>>
>>>>>>>>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>> wrote:
>>>>>>>>>>
>>>>>>>>>>> Yea I really like that idea I'll see what I can do to update
>>> the
>>>>> kip
>>>>>>>>> and
>>>>>>>>>>> my pr when I have some time. I'm not sure how well creating
>> the
>>>>>>>>>>> kstreamaggregates will go though because at that point I will
>>>>> have
>>>>>>>>> thrown
>>>>>>>>>>> away the type of the values. It will be type safe I just may
>>>>> need to
>>>>>>>>> do a
>>>>>>>>>>> little forcing.
>>>>>>>>>>>
>>>>>>>>>>> Thanks,
>>>>>>>>>>> Kyle
>>>>>>>>>>>
>>>>>>>>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
>>>>>> wrote:
>>>>>>>>>>>> Kyle,
>>>>>>>>>>>>
>>>>>>>>>>>> Thanks for the explanations, my previous read on the wiki
>>>>> examples
>>>>>>> was
>>>>>>>>>>>> wrong.
>>>>>>>>>>>>
>>>>>>>>>>>> So I guess my motivation should be "reduced" to: can we move
>>> the
>>>>>>>>> window
>>>>>>>>>>>> specs param from "KGroupedStream#cogroup(..)" to
>>>>>>>>>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>>>>>>>>>
>>>>>>>>>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
>>> from
>>>>> 3
>>>>>> to
>>>>>>>>> 2.
>>>>>>>>>>>> 2. major: this is for extensibility of the APIs, and since
>> we
>>>>> are
>>>>>>>>>> removing
>>>>>>>>>>>> the "Evolving" annotations on Streams it may be harder to
>>>>> change it
>>>>>>>>>> again
>>>>>>>>>>>> in the future. The extended use cases are that people wanted
>>> to
>>>>>> have
>>>>>>>>>>>> windowed running aggregates on different granularities, e.g.
>>>>> "give
>>>>>> me
>>>>>>>>>> the
>>>>>>>>>>>> counts per-minute, per-hour, per-day and per-week", and
>> today
>>> in
>>>>>> DSL
>>>>>>>>> we
>>>>>>>>>>>> need to specify that case in multiple aggregate operators,
>>> which
>>>>>> gets
>>>>>>>>> a
>>>>>>>>>>>> state store / changelog, etc. And it is possible to optimize
>>> it
>>>>> as
>>>>>>>>> well
>>>>>>>>>> to
>>>>>>>>>>>> a single state store. Its implementation would be tricky as
>>> you
>>>>>> need
>>>>>>>>> to
>>>>>>>>>>>> contain different lengthed windows within your window store
>>> but
>>>>>> just
>>>>>>>>>> from
>>>>>>>>>>>> the public API point of view, it could be specified as:
>>>>>>>>>>>>
>>>>>>>>>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>>>>>>>>> "state-store-name");
>>>>>>>>>>>>
>>>>>>>>>>>> table1 = stream.aggregate(/*per-minute window*/)
>>>>>>>>>>>> table2 = stream.aggregate(/*per-hour window*/)
>>>>>>>>>>>> table3 = stream.aggregate(/*per-day window*/)
>>>>>>>>>>>>
>>>>>>>>>>>> while underlying we are only using a single store
>>>>>> "state-store-name"
>>>>>>>>> for
>>>>>>>>>>>> it.
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> Although this feature is out of the scope of this KIP, I'd
>>> like
>>>>> to
>>>>>>>>>> discuss
>>>>>>>>>>>> if we can "leave the door open" to make such changes without
>>>>>>> modifying
>>>>>>>>>> the
>>>>>>>>>>>> public APIs .
>>>>>>>>>>>>
>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>
>>>>>>>>>>>>> I allow defining a single window/sessionwindow one time
>> when
>>>>> you
>>>>>>>>> make
>>>>>>>>>>>> the
>>>>>>>>>>>>> cogroup call from a KGroupedStream. From then on you are
>>> using
>>>>> the
>>>>>>>>>>>> cogroup
>>>>>>>>>>>>> call from with in CogroupedKStream which doesnt accept any
>>>>>>>>> additional
>>>>>>>>>>>>> windows/sessionwindows.
>>>>>>>>>>>>>
>>>>>>>>>>>>> Is this what you meant by your question or did I
>>> misunderstand?
>>>>>>>>>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
>> wangguoz@gmail.com
>>>>>>>>> wrote:
>>>>>>>>>>>>> Another question that came to me is on "window alignment":
>>> from
>>>>>> the
>>>>>>>>>> KIP
>>>>>>>>>>>> it
>>>>>>>>>>>>> seems you are allowing users to specify a (potentially
>>>>> different)
>>>>>>>>>> window
>>>>>>>>>>>>> spec in each co-grouped input stream. So if these window
>>> specs
>>>>> are
>>>>>>>>>>>>> different how should we "align" them with different input
>>>>>> streams? I
>>>>>>>>>>>> think
>>>>>>>>>>>>> it is more natural to only specify on window spec in the
>>>>>>>>>>>>>
>>>>>>>>>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> And remove it from the cogroup() functions. WDYT?
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>>>>>> wangguoz@gmail.com>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> Thanks for the proposal Kyle, this is a quite common use
>>> case
>>>>> to
>>>>>>>>>>>> support
>>>>>>>>>>>>>> such multi-way table join (i.e. N source tables with N
>>>>> aggregate
>>>>>>>>>> func)
>>>>>>>>>>>>> with
>>>>>>>>>>>>>> a single store and N+1 serdes, I have seen lots of people
>>>>> using
>>>>>>>>> the
>>>>>>>>>>>>>> low-level PAPI to achieve this goal.
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>> I like your point about not handling other cases such as
>>>>> count
>>>>>>>>> and
>>>>>>>>>>>>> reduce.
>>>>>>>>>>>>>>> I think that reduce may not make sense because reduce
>>> assumes
>>>>>>>>> that
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> input values are the same as the output values. With
>>> cogroup
>>>>>>>>> there
>>>>>>>>>>>> may
>>>>>>>>>>>>> be
>>>>>>>>>>>>>>> multiple different input types and then your output type
>>>>> cant be
>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>> different things. In the case where you have all matching
>>>>> value
>>>>>>>>>> types
>>>>>>>>>>>>> you
>>>>>>>>>>>>>>> can do KStreamBuilder#merge followed by the reduce.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> As for count I think it is possible to call count on all
>>> the
>>>>>>>>>>>> individual
>>>>>>>>>>>>>>> grouped streams and then do joins. Otherwise we could
>> maybe
>>>>> make
>>>>>>>>> a
>>>>>>>>>>>>> special
>>>>>>>>>>>>>>> call in groupedstream for this case. Because in this case
>>> we
>>>>>> dont
>>>>>>>>>>>> need
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>> do type checking on the values. It could be similar to
>> the
>>>>>>>>> current
>>>>>>>>>>>> count
>>>>>>>>>>>>>>> methods but accept a var args of additonal grouped
>> streams
>>> as
>>>>>>>>> well
>>>>>>>>>>>> and
>>>>>>>>>>>>>>> make
>>>>>>>>>>>>>>> sure they have a key type of K.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> The way I have put the kip together is to ensure that we
>> do
>>>>> type
>>>>>>>>>>>>> checking.
>>>>>>>>>>>>>>> I don't see a way we could group them all first and then
>>>>> make a
>>>>>>>>>> call
>>>>>>>>>>>> to
>>>>>>>>>>>>>>> count, reduce, or aggregate because with aggregate they
>>> would
>>>>>>>>> need
>>>>>>>>>> to
>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>> a list of aggregators and we would have no way of type
>>>>> checking
>>>>>>>>>> that
>>>>>>>>>>>>> they
>>>>>>>>>>>>>>> match the grouped streams.
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>> Kyle
>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
>>>>> xavier@confluent.io>
>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> Sorry to jump on this thread so late. I agree this is a
>>> very
>>>>>>>>>> useful
>>>>>>>>>>>>>>>> addition and wanted to provide an additional use-case
>> and
>>>>> some
>>>>>>>>>> more
>>>>>>>>>>>>>>>> comments.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> This is actually a very common analytics use-case in the
>>>>>>>>> ad-tech
>>>>>>>>>>>>>>> industry.
>>>>>>>>>>>>>>>> The typical setup will have an auction stream, an
>>> impression
>>>>>>>>>>>> stream,
>>>>>>>>>>>>>>> and a
>>>>>>>>>>>>>>>> click stream. Those three streams need to be combined to
>>>>>>>>> compute
>>>>>>>>>>>>>>> aggregate
>>>>>>>>>>>>>>>> statistics (e.g. impression statistics, and
>> click-through
>>>>>>>>> rates),
>>>>>>>>>>>>> since
>>>>>>>>>>>>>>>> most of the attributes of interest are only present the
>>>>> auction
>>>>>>>>>>>>> stream.
>>>>>>>>>>>>>>>> A simple way to do this is to co-group all the streams
>> by
>>>>> the
>>>>>>>>>>>> auction
>>>>>>>>>>>>>>> key,
>>>>>>>>>>>>>>>> and process updates to the co-group as events for each
>>>>> stream
>>>>>>>>>> come
>>>>>>>>>>>> in,
>>>>>>>>>>>>>>>> keeping only one value from each stream before sending
>>>>>>>>> downstream
>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> further processing / aggregation.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> One could view the result of that co-group operation as
>> a
>>>>>>>>>> "KTable"
>>>>>>>>>>>>> with
>>>>>>>>>>>>>>>> multiple values per key. The key being the grouping key,
>>> and
>>>>>>>>> the
>>>>>>>>>>>>> values
>>>>>>>>>>>>>>>> consisting of one value per stream.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> What I like about Kyle's approach is that allows elegant
>>>>>>>>>>>> co-grouping
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> multiple streams without having to worry about the
>> number
>>> of
>>>>>>>>>>>> streams,
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>> avoids dealing with Tuple types or other generic
>>> interfaces
>>>>>>>>> that
>>>>>>>>>>>> could
>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>> messy if we wanted to preserve all the value types in
>> the
>>>>>>>>>> resulting
>>>>>>>>>>>>>>>> co-grouped stream.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> My only concern is that we only allow the cogroup +
>>>>> aggregate
>>>>>>>>>>>> combined
>>>>>>>>>>>>>>>> operation. This forces the user to build their own tuple
>>>>>>>>>>>> serialization
>>>>>>>>>>>>>>>> format if they want to preserve the individual input
>>> stream
>>>>>>>>>> values
>>>>>>>>>>>> as
>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>> group. It also deviates quite a bit from our approach in
>>>>>>>>>>>>> KGroupedStream
>>>>>>>>>>>>>>>> which offers other operations, such as count and reduce,
>>>>> which
>>>>>>>>>>>> should
>>>>>>>>>>>>>>> also
>>>>>>>>>>>>>>>> be applicable to a co-grouped stream.
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>> Overall I still think this is a really useful addition,
>>> but
>>>>> I
>>>>>>>>>> feel
>>>>>>>>>>>> we
>>>>>>>>>>>>>>>> haven't spend much time trying to explore alternative
>> DSLs
>>>>> that
>>>>>>>>>>>> could
>>>>>>>>>>>>>>> maybe
>>>>>>>>>>>>>>>> generalize better or match our existing syntax more
>>> closely.
>>>>>>>>>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> Eno, is there anyone else that is an expert in the
>> kafka
>>>>>>>>>> streams
>>>>>>>>>>>>> realm
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>> I should reach out to for input?
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> I believe Damian Guy is still planning on reviewing
>> this
>>>>> more
>>>>>>>>>> in
>>>>>>>>>>>>> depth
>>>>>>>>>>>>>>>> so I
>>>>>>>>>>>>>>>>> will wait for his inputs before continuing.
>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
>>>>>>>>> eno.thereska@gmail.com
>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>> Thanks Kyle, good arguments.
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>> Eno
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> *- minor: could you add an exact example (similar to
>>> what
>>>>>>>>>>>> Jay’s
>>>>>>>>>>>>>>>> example
>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
>> super
>>>>>>>>>>>>> concrete?*
>>>>>>>>>>>>>>>>>>> I have added a more concrete example to the KIP.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> *- my main concern is that we’re exposing this
>>>>>>>>> optimization
>>>>>>>>>>>> to
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> DSL.
>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>> an ideal world, an optimizer would take the existing
>>> DSL
>>>>>>>>>> and
>>>>>>>>>>>> do
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>> thing under the covers (create just one state store,
>>>>>>>>>> arrange
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> nodes
>>>>>>>>>>>>>>>>>>> etc). The original DSL had a bunch of small,
>> composable
>>>>>>>>>>>> pieces
>>>>>>>>>>>>>>>> (group,
>>>>>>>>>>>>>>>>>>> aggregate, join) that this proposal groups together.
>>> I’d
>>>>>>>>>>>> like to
>>>>>>>>>>>>>>> hear
>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>> thoughts on whether it’s possible to do this
>>> optimization
>>>>>>>>>>>> with
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>> DSL, at the topology builder level.*
>>>>>>>>>>>>>>>>>>> You would have to make a lot of checks to understand
>> if
>>>>>>>>> it
>>>>>>>>>> is
>>>>>>>>>>>>> even
>>>>>>>>>>>>>>>>>> possible
>>>>>>>>>>>>>>>>>>> to make this optimization:
>>>>>>>>>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
>>>>>>>>>>>>>>>>>>> 2. None of the intermediate KTables are used for
>>> anything
>>>>>>>>>>>> else.
>>>>>>>>>>>>>>>>>>> 3. None of the intermediate stores are used. (This
>> may
>>> be
>>>>>>>>>>>>>>> impossible
>>>>>>>>>>>>>>>>>>> especially if they use KafkaStreams#store after the
>>>>>>>>>> topology
>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>> been built.)
>>>>>>>>>>>>>>>>>>> You would then need to make decisions during the
>>>>>>>>>>>> optimization:
>>>>>>>>>>>>>>>>>>> 1. Your new initializer would the composite of all
>> the
>>>>>>>>>>>>> individual
>>>>>>>>>>>>>>>>>>> initializers and the valueJoiners.
>>>>>>>>>>>>>>>>>>> 2. I am having a hard time thinking about how you
>> would
>>>>>>>>>> turn
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
>>>>>>>>> would
>>>>>>>>>>>> work
>>>>>>>>>>>>> on
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> final object, but this may be possible.
>>>>>>>>>>>>>>>>>>> 3. Which state store would you use? The ones declared
>>>>>>>>> would
>>>>>>>>>>>> be
>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> aggregate values. None of the declared ones would be
>>>>>>>>>>>> guaranteed
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> hold
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> final object. This would mean you must created a new
>>>>>>>>> state
>>>>>>>>>>>> store
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>> created any of the declared ones.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> The main argument I have against it is even if it
>> could
>>>>>>>>> be
>>>>>>>>>>>> done
>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>>> know that we would want to have this be an
>> optimization
>>>>>>>>> in
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> background
>>>>>>>>>>>>>>>>>>> because the user would still be required to think
>> about
>>>>>>>>> all
>>>>>>>>>>>> of
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> intermediate values that they shouldn't need to worry
>>>>>>>>> about
>>>>>>>>>>>> if
>>>>>>>>>>>>>>> they
>>>>>>>>>>>>>>>>> only
>>>>>>>>>>>>>>>>>>> care about the final object.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> In my opinion cogroup is a common enough case that it
>>>>>>>>>> should
>>>>>>>>>>>> be
>>>>>>>>>>>>>>> part
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> composable pieces (group, aggregate, join) because we
>>>>>>>>> want
>>>>>>>>>> to
>>>>>>>>>>>>>>> allow
>>>>>>>>>>>>>>>>>> people
>>>>>>>>>>>>>>>>>>> to join more than 2 or more streams in an easy way.
>>> Right
>>>>>>>>>>>> now I
>>>>>>>>>>>>>>> don't
>>>>>>>>>>>>>>>>>> think
>>>>>>>>>>>>>>>>>>> we give them ways of handling this use case easily.
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> *-I think there will be scope for several such
>>>>>>>>>> optimizations
>>>>>>>>>>>> in
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> future
>>>>>>>>>>>>>>>>>>> and perhaps at some point we need to think about
>>>>>>>>> decoupling
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> 1:1
>>>>>>>>>>>>>>>>>> mapping
>>>>>>>>>>>>>>>>>>> from the DSL into the physical topology.*
>>>>>>>>>>>>>>>>>>> I would argue that cogroup is not just an
>> optimization
>>> it
>>>>>>>>>> is
>>>>>>>>>>>> a
>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>> way
>>>>>>>>>>>>>>>>>> for
>>>>>>>>>>>>>>>>>>> the users to look at accomplishing a problem that
>>>>>>>>> requires
>>>>>>>>>>>>>>> multiple
>>>>>>>>>>>>>>>>>>> streams. I may sound like a broken record but I don't
>>>>>>>>> think
>>>>>>>>>>>>> users
>>>>>>>>>>>>>>>>> should
>>>>>>>>>>>>>>>>>>> have to build the N-1 intermediate tables and deal
>> with
>>>>>>>>>> their
>>>>>>>>>>>>>>>>>> initializers,
>>>>>>>>>>>>>>>>>>> serdes and stores if all they care about is the final
>>>>>>>>>> object.
>>>>>>>>>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
>>>>>>>>> supply
>>>>>>>>>>>>>>> additional
>>>>>>>>>>>>>>>>>>> streams and aggregators this case is equivalent to a
>>>>>>>>> single
>>>>>>>>>>>>>>> grouped
>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>> making an aggregate call. This case is what I view an
>>>>>>>>>>>>> optimization
>>>>>>>>>>>>>>>> as,
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>> could remove the KStreamCogroup and act as if there
>> was
>>>>>>>>>> just
>>>>>>>>>>>> a
>>>>>>>>>>>>>>> call
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> KGroupedStream#aggregate instead of calling
>>>>>>>>>>>>>>> KGroupedStream#cogroup.
>>>>>>>>>>>>>>>> (I
>>>>>>>>>>>>>>>>>>> would prefer to just write a warning saying that this
>>> is
>>>>>>>>>> not
>>>>>>>>>>>> how
>>>>>>>>>>>>>>>>> cogroup
>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>> to be used.)
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Kyle
>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
>>>>>>>>>>>>>>> eno.thereska@gmail.com
>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> - minor: could you add an exact example (similar to
>>> what
>>>>>>>>>>>> Jay’s
>>>>>>>>>>>>>>>> example
>>>>>>>>>>>>>>>>>> is,
>>>>>>>>>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
>>> super
>>>>>>>>>>>>> concrete?
>>>>>>>>>>>>>>>>>>>> - my main concern is that we’re exposing this
>>>>>>>>> optimization
>>>>>>>>>>>> to
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> DSL.
>>>>>>>>>>>>>>>>>> In
>>>>>>>>>>>>>>>>>>>> an ideal world, an optimizer would take the existing
>>> DSL
>>>>>>>>>>>> and do
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> right
>>>>>>>>>>>>>>>>>>>> thing under the covers (create just one state store,
>>>>>>>>>> arrange
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> nodes
>>>>>>>>>>>>>>>>>>>> etc). The original DSL had a bunch of small,
>>> composable
>>>>>>>>>>>> pieces
>>>>>>>>>>>>>>>> (group,
>>>>>>>>>>>>>>>>>>>> aggregate, join) that this proposal groups together.
>>> I’d
>>>>>>>>>>>> like
>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> hear
>>>>>>>>>>>>>>>>>> your
>>>>>>>>>>>>>>>>>>>> thoughts on whether it’s possible to do this
>>>>>>>>> optimization
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> DSL, at the topology builder level.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> I think there will be scope for several such
>>>>>>>>> optimizations
>>>>>>>>>>>> in
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> future
>>>>>>>>>>>>>>>>>>>> and perhaps at some point we need to think about
>>>>>>>>>> decoupling
>>>>>>>>>>>> the
>>>>>>>>>>>>>>> 1:1
>>>>>>>>>>>>>>>>>> mapping
>>>>>>>>>>>>>>>>>>>> from the DSL into the physical topology.
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>>>>>>>>> Eno
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
>>>>>>>>> jay@confluent.io>
>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> I haven't digested the proposal but the use case is
>>>>>>>>>> pretty
>>>>>>>>>>>>>>> common.
>>>>>>>>>>>>>>>> An
>>>>>>>>>>>>>>>>>>>>> example would be the "customer 360" or "unified
>>>>>>>>> customer
>>>>>>>>>>>>>>> profile"
>>>>>>>>>>>>>>>> use
>>>>>>>>>>>>>>>>>>>> case
>>>>>>>>>>>>>>>>>>>>> we often use. In that use case you have a dozen
>>> systems
>>>>>>>>>>>> each
>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>> some information about your customer (account
>>> details,
>>>>>>>>>>>>> settings,
>>>>>>>>>>>>>>>>>> billing
>>>>>>>>>>>>>>>>>>>>> info, customer service contacts, purchase history,
>>>>>>>>> etc).
>>>>>>>>>>>> Your
>>>>>>>>>>>>>>> goal
>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>> join/munge these into a single profile record for
>>> each
>>>>>>>>>>>>> customer
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>> has
>>>>>>>>>>>>>>>>>>>>> all the relevant info in a usable form and is
>>>>>>>>> up-to-date
>>>>>>>>>>>> with
>>>>>>>>>>>>>>> all
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> source systems. If you implement that with kstreams
>>> as
>>>>>>>>> a
>>>>>>>>>>>>>>> sequence
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>> joins
>>>>>>>>>>>>>>>>>>>>> i think today we'd fully materialize N-1
>> intermediate
>>>>>>>>>>>> tables.
>>>>>>>>>>>>>>> But
>>>>>>>>>>>>>>>>>> clearly
>>>>>>>>>>>>>>>>>>>>> you only need a single stage to group all these
>>> things
>>>>>>>>>> that
>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>> co-partitioned. A distributed database would do
>> this
>>>>>>>>>> under
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> covers
>>>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>> is arguably better (at least when it does the right
>>>>>>>>>> thing)
>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> perhaps
>>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>>> could do the same thing but I'm not sure we know
>> the
>>>>>>>>>>>>>>> partitioning
>>>>>>>>>>>>>>>> so
>>>>>>>>>>>>>>>>> we
>>>>>>>>>>>>>>>>>>>> may
>>>>>>>>>>>>>>>>>>>>> need an explicit cogroup command that impllies they
>>> are
>>>>>>>>>>>>> already
>>>>>>>>>>>>>>>>>>>>> co-partitioned.
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
>>>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> Yea thats a good way to look at it.
>>>>>>>>>>>>>>>>>>>>>> I have seen this type of functionality in a couple
>>>>>>>>> other
>>>>>>>>>>>>>>> platforms
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> spark and pig.
>>>>>>>>>>>>>>>>>>>>>> https://spark.apache.org/docs/
>> 0.6.2/api/core/spark/
>>>>>>>>>>>>>>>>>>>> PairRDDFunctions.html
>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
>>>>>>>>>>>>>>>>>>>> cogroup_operator.htm
>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
>>>>>>>>> outer
>>>>>>>>>>>>> join?
>>>>>>>>>>>>>>> So
>>>>>>>>>>>>>>>> an
>>>>>>>>>>>>>>>>>>>> input
>>>>>>>>>>>>>>>>>>>>>>> on any stream will always produce a new
>> aggregated
>>>>>>>>>> value
>>>>>>>>>>>> -
>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>> that
>>>>>>>>>>>>>>>>>>>>>> correct?
>>>>>>>>>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
>>>>>>>>> current
>>>>>>>>>>>>> value,
>>>>>>>>>>>>>>>>>>>> aggregates
>>>>>>>>>>>>>>>>>>>>>>> and forwards the result.
>>>>>>>>>>>>>>>>>>>>>>> I need to look into it and think about it a bit
>>> more,
>>>>>>>>>>>> but it
>>>>>>>>>>>>>>>> seems
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> could be a useful optimization.
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
>>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> I sure can. I have added the following
>> description
>>>>>>>>> to
>>>>>>>>>> my
>>>>>>>>>>>>>>> KIP. If
>>>>>>>>>>>>>>>>>> this
>>>>>>>>>>>>>>>>>>>>>>>> doesn't help let me know and I will take some
>> more
>>>>>>>>>> time
>>>>>>>>>>>> to
>>>>>>>>>>>>>>>> build a
>>>>>>>>>>>>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>>>>>>>>> and make more of a step by step description:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Example with Current API:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
>>>>>>>>>>>>>>>>>>>>>>>> builder.stream("topic1").group
>> ByKey().aggregate(
>>>>>>>>>>>>> initializer1
>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>> aggregator1,
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
>>>>>>>>>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
>>>>>>>>>>>>>>>>>>>>>>>> builder.stream("topic2").group
>> ByKey().aggregate(
>>>>>>>>>>>>> initializer2
>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>> aggregator2,
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
>>>>>>>>>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
>>>>>>>>>>>>>>>>>>>>>>>> builder.stream("topic3").group
>> ByKey().aggregate(
>>>>>>>>>>>>> initializer3
>>>>>>>>>>>>>>> ,
>>>>>>>>>>>>>>>>>>>>>>> aggregator3,
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
>>>>>>>>>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
>> table1.outerJoin(table2,
>>>>>>>>>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
>>>>>>>>>>>> joinerOneTwoAndThree);
>>>>>>>>>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores,
>>> requires
>>>>>>>>> 3
>>>>>>>>>>>>>>>>> initializers,
>>>>>>>>>>>>>>>>>>>>>> and 3
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to
>>> user
>>>>>>>>> to
>>>>>>>>>>>>> define
>>>>>>>>>>>>>>>> what
>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2,
>> V3).
>>>>>>>>> They
>>>>>>>>>>>> are
>>>>>>>>>>>>>>> left
>>>>>>>>>>>>>>>>>> with a
>>>>>>>>>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all
>>> the
>>>>>>>>>>>> same
>>>>>>>>>>>>> as
>>>>>>>>>>>>>>> CG
>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>>>>>>>>> joiners are more like mergers, or second make
>> them
>>>>>>>>>>>>>>> intermediate
>>>>>>>>>>>>>>>>>> states
>>>>>>>>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
>>>>>>>>> joiners
>>>>>>>>>>>> use
>>>>>>>>>>>>>>> those
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>>>>>>>>> the final aggregate CG value. This is something
>>> the
>>>>>>>>>> user
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>>>>>>>>> thinking about with this KIP.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>>>>>>>>> will
>>>>>>>>>>>> first
>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate
>>> from
>>>>>>>>>>>>>>> storeName1.
>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>> produce this in the form of the first
>> intermediate
>>>>>>>>>> value
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will
>> look
>>>>>>>>> up
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> current
>>>>>>>>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>>>>>>>>> the key in storeName2. It will use the first
>>> joiner
>>>>>>>>> to
>>>>>>>>>>>>>>> calculate
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>>>>>>>>> intermediate value, which will go through an
>>>>>>>>>> additional
>>>>>>>>>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
>>>>>>>>>> current
>>>>>>>>>>>>>>> value of
>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build
>> the
>>>>>>>>>> final
>>>>>>>>>>>>>>>> aggregate
>>>>>>>>>>>>>>>>>>>>>> value.
>>>>>>>>>>>>>>>>>>>>>>>> If you think through all possibilities for
>>> incoming
>>>>>>>>>>>> topics
>>>>>>>>>>>>>>> you
>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>>>>>>>>> that no matter which topic it comes in through
>> all
>>>>>>>>>> three
>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>> are
>>>>>>>>>>>>>>>>>>>>>>> queried
>>>>>>>>>>>>>>>>>>>>>>>> and all of the joiners must get used.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Topology wise for N incoming streams this
>> creates
>>> N
>>>>>>>>>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1)
>> KTableKTableOuterJoins,
>>>>>>>>> and
>>>>>>>>>>>> N-1
>>>>>>>>>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> Example with Proposed API:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
>>>>>>>>>>>> builder.stream("topic1").
>>>>>>>>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
>>>>>>>>>>>> builder.stream("topic2").
>>>>>>>>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
>>>>>>>>>>>> builder.stream("topic3").
>>>>>>>>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
>>>>>>>>>> grouped1.cogroup(initializer1,
>>>>>>>>>>>>>>>>>> aggregator1,
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
>>>>>>>>>>>>>>>>>>>>>>>>       .cogroup(grouped2, aggregator2)
>>>>>>>>>>>>>>>>>>>>>>>>       .cogroup(grouped3, aggregator3)
>>>>>>>>>>>>>>>>>>>>>>>>       .aggregate();
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore,
>>> requires 1
>>>>>>>>>>>>>>>> initializer,
>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>> 1
>>>>>>>>>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry
>>> about
>>>>>>>>>> the
>>>>>>>>>>>>>>>>>> intermediate
>>>>>>>>>>>>>>>>>>>>>>>> values and the joiners. All they have to think
>>> about
>>>>>>>>>> is
>>>>>>>>>>>> how
>>>>>>>>>>>>>>> each
>>>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>>>>>>>>> will
>>>>>>>>>>>> first
>>>>>>>>>>>>>>> go
>>>>>>>>>>>>>>>>>> through
>>>>>>>>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate
>>> from
>>>>>>>>>>>>>>> storeName1.
>>>>>>>>>>>>>>>>> It
>>>>>>>>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update
>>> the
>>>>>>>>>>>> store
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> pass
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through
>> the
>>>>>>>>>>>>>>> KStreamCogroup
>>>>>>>>>>>>>>>>>> which
>>>>>>>>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>>>>>>>>> pretty much just a pass through processor and
>> you
>>>>>>>>> are
>>>>>>>>>>>> done.
>>>>>>>>>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
>>>>>>>>> will
>>>>>>>>>>>> only
>>>>>>>>>>>>>>> every
>>>>>>>>>>>>>>>>>>>>>> create N
>>>>>>>>>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax
>> <
>>>>>>>>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Kyle,
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little
>>> slow,
>>>>>>>>>>>> but I
>>>>>>>>>>>>>>> could
>>>>>>>>>>>>>>>>> not
>>>>>>>>>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
>>>>>>>>>> concrete
>>>>>>>>>>>>>>> example,
>>>>>>>>>>>>>>>>>> like
>>>>>>>>>>>>>>>>>>>>>> 3
>>>>>>>>>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected
>>> result),
>>>>>>>>>> and
>>>>>>>>>>>>> show
>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>> difference between current way to to implement
>> it
>>>>>>>>> and
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> proposed
>>>>>>>>>>>>>>>>>>>>>> API?
>>>>>>>>>>>>>>>>>>>>>>>>> This could also cover the internal processing
>> to
>>>>>>>>> see
>>>>>>>>>>>> what
>>>>>>>>>>>>>>> store
>>>>>>>>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>>>>>>>>> would be required for both approaches etc.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you
>> propose,
>>>>>>>>> and
>>>>>>>>>> it
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>>> help
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>> understand it better.
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> Thanks a lot!
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found
>>> here.
>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
>>>>>>>>> classes
>>>>>>>>>>>> and
>>>>>>>>>>>>> get
>>>>>>>>>>>>>>>>> around
>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>> writing documentation for the public api
>>>>>>>>> additions.
>>>>>>>>>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
>>>>>>>>>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
>>>>>>>>>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
>>>>>>>>>>>>>>>> repartitionIfRequired
>>>>>>>>>>>>>>>>>>>>>>> method.
>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>> can't supply the store name because if more
>> than
>>>>>>>>> one
>>>>>>>>>>>>>>> grouped
>>>>>>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
>>>>>>>>> name
>>>>>>>>>>>> that
>>>>>>>>>>>>>>>> someone
>>>>>>>>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow
>>> it
>>>>>>>>> to
>>>>>>>>>>>> fall
>>>>>>>>>>>>>>> back
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>> KGroupedStream.name?
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped
>>> tables?
>>>>>>>>>> This
>>>>>>>>>>>>>>> would
>>>>>>>>>>>>>>>> be
>>>>>>>>>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>>>>>>>>> easy
>>>>>>>>>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing
>> session
>>>>>>>>>> stores
>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> windowed
>>>>>>>>>>>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate
>> and
>>>>>>>>>>>>>>>>>>>>>> KTableWindowAggregate
>>>>>>>>>>>>>>>>>>>>>>>>>> implementations.
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>> Kyle
>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
>>>>>>>>>>>>>>>> eno.thereska@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>> Eno
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
>>>>>>>>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i
>> haven't
>>>>>>>>> had
>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> chance
>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to
>>> look
>>>>>>>>>>>> into
>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>> tomorrow.
>>>>>>>>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against
>>> kafka
>>>>>>>>>>>> trunk
>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>> mark
>>>>>>>>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>> WIP?
>>>>>>>>>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have
>>> done.
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
>>>>>>>>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
>>>>>>>>> some
>>>>>>>>>>>>>>> attention
>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
>>>>>>>>> This
>>>>>>>>>>>> is my
>>>>>>>>>>>>>>>> first
>>>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so
>>> I'm
>>>>>>>>>>>> sure I
>>>>>>>>>>>>>>> did
>>>>>>>>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrong.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> ;)
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
>>>>>>>>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hello all,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
>>>>>>>>> discussion
>>>>>>>>>>>> about
>>>>>>>>>>>>>>>> adding
>>>>>>>>>>>>>>>>>>>>>>>> cogroup
>>>>>>>>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>>>>>>>>> confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation
>> here:
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
>>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>>
>>>>>>>>>>>>>> --
>>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>>
>>>>>>>>>>>>> --
>>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>> --
>>>>>>>>>>>> -- Guozhang
>>>>>>>>>>>>
>>>>>>>
>>>>>>
>>>>>> --
>>>>>> -- Guozhang
>>>>>>
>>>>
>>>>
>>>> --
>>>> -- Guozhang
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>
>

-- 
Signature
<http://www.openbet.com/> 	Michal Borowiecki
Senior Software Engineer L4
	T: 	+44 208 742 1600

	
	+44 203 249 8448

	
	
	E: 	michal.borowiecki@openbet.com
	W: 	www.openbet.com <http://www.openbet.com/>

	
	OpenBet Ltd

	Chiswick Park Building 9

	566 Chiswick High Rd

	London

	W4 5XT

	UK

	
<https://www.openbet.com/email_promo>

This message is confidential and intended only for the addressee. If you 
have received this message in error, please immediately notify the 
postmaster@openbet.com <ma...@openbet.com> and delete it 
from your system as well as any copies. The content of e-mails as well 
as traffic data may be monitored by OpenBet for employment and security 
purposes. To protect the environment please do not print this e-mail 
unless necessary. OpenBet Ltd. Registered Office: Chiswick Park Building 
9, 566 Chiswick High Road, London, W4 5XT, United Kingdom. A company 
registered in England and Wales. Registered no. 3134634. VAT no. 
GB927523612


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
Note that although the internal `AbstractStoreSupplier` does maintain the
key-value serdes, we do not enforce the interface of `StateStoreSupplier`
to always retain that information, and hence we cannot assume that
StateStoreSuppliers always retain key / value serdes.

On Thu, Jun 8, 2017 at 11:51 AM, Xavier Léauté <xa...@confluent.io> wrote:

> Another reason for the serde not to be in the first cogroup call, is that
> the serde should not be required if you pass a StateStoreSupplier to
> aggregate()
>
> Regarding the aggregated type <T> I don't the why initializer should be
> favored over aggregator to define the type. In my mind separating the
> initializer into the last aggregate call clearly indicates that the
> initializer is independent of any of the aggregators or streams and that we
> don't wait for grouped1 events to initialize the co-group.
>
> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > On a second thought... This is the current proposal API
> >
> >
> > ```
> >
> > <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
> final
> > Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> > aggValueSerde)
> >
> > ```
> >
> >
> > If we do not have the initializer in the first co-group it might be a bit
> > awkward for users to specify the aggregator that returns a typed <T>
> value?
> > Maybe it is still better to put these two functions in the same api?
> >
> >
> >
> > Guozhang
> >
> > On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > This suggestion lgtm. I would vote for the first alternative than
> adding
> > > it to the `KStreamBuilder` though.
> > >
> > > On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
> > > wrote:
> > >
> > >> I have a minor suggestion to make the API a little bit more symmetric.
> > >> I feel it would make more sense to move the initializer and serde to
> the
> > >> final aggregate statement, since the serde only applies to the state
> > >> store,
> > >> and the initializer doesn't bear any relation to the first group in
> > >> particular. It would end up looking like this:
> > >>
> > >> KTable<K, CG> cogrouped =
> > >>     grouped1.cogroup(aggregator1)
> > >>             .cogroup(grouped2, aggregator2)
> > >>             .cogroup(grouped3, aggregator3)
> > >>             .aggregate(initializer1, aggValueSerde, storeName1);
> > >>
> > >> Alternatively, we could move the the first cogroup() method to
> > >> KStreamBuilder, similar to how we have .merge()
> > >> and end up with an api that would be even more symmetric.
> > >>
> > >> KStreamBuilder.cogroup(grouped1, aggregator1)
> > >>               .cogroup(grouped2, aggregator2)
> > >>               .cogroup(grouped3, aggregator3)
> > >>               .aggregate(initializer1, aggValueSerde, storeName1);
> > >>
> > >> This doesn't have to be a blocker, but I thought it would make the API
> > >> just
> > >> a tad cleaner.
> > >>
> > >> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>
> > >> > Kyle,
> > >> >
> > >> > Thanks a lot for the updated KIP. It looks good to me.
> > >> >
> > >> >
> > >> > Guozhang
> > >> >
> > >> >
> > >> > On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> > wrote:
> > >> >
> > >> > > This makes much more sense to me. +1
> > >> > >
> > >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >> winkelman.kyle@gmail.com>
> > >> > > wrote:
> > >> > > >
> > >> > > > I have updated the KIP and my PR. Let me know what you think.
> > >> > > > To created a cogrouped stream just call cogroup on a
> > KgroupedStream
> > >> and
> > >> > > > supply the initializer, aggValueSerde, and an aggregator. Then
> > >> continue
> > >> > > > adding kgroupedstreams and aggregators. Then call one of the
> many
> > >> > > aggregate
> > >> > > > calls to create a KTable.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Kyle
> > >> > > >
> > >> > > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com>
> > wrote:
> > >> > > >
> > >> > > >> Hi Kyle,
> > >> > > >>
> > >> > > >> Thanks for the update. I think just one initializer makes sense
> > as
> > >> it
> > >> > > >> should only be called once per key and generally it is just
> going
> > >> to
> > >> > > create
> > >> > > >> a new instance of whatever the Aggregate class is.
> > >> > > >>
> > >> > > >> Cheers,
> > >> > > >> Damian
> > >> > > >>
> > >> > > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > >> winkelman.kyle@gmail.com
> > >> > >
> > >> > > >> wrote:
> > >> > > >>
> > >> > > >>> Hello all,
> > >> > > >>>
> > >> > > >>> I have spent some more time on this and the best alternative I
> > >> have
> > >> > > come
> > >> > > >> up
> > >> > > >>> with is:
> > >> > > >>> KGroupedStream has a single cogroup call that takes an
> > initializer
> > >> > and
> > >> > > an
> > >> > > >>> aggregator.
> > >> > > >>> CogroupedKStream has a cogroup call that takes additional
> > >> > groupedStream
> > >> > > >>> aggregator pairs.
> > >> > > >>> CogroupedKStream has multiple aggregate methods that create
> the
> > >> > > different
> > >> > > >>> stores.
> > >> > > >>>
> > >> > > >>> I plan on updating the kip but I want people's input on if we
> > >> should
> > >> > > have
> > >> > > >>> the initializer be passed in once at the beginning or if we
> > should
> > >> > > >> instead
> > >> > > >>> have the initializer be required for each call to one of the
> > >> > aggregate
> > >> > > >>> calls. The first makes more sense to me but doesnt allow the
> > user
> > >> to
> > >> > > >>> specify different initializers for different tables.
> > >> > > >>>
> > >> > > >>> Thanks,
> > >> > > >>> Kyle
> > >> > > >>>
> > >> > > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > >> winkelman.kyle@gmail.com>
> > >> > > >>> wrote:
> > >> > > >>>
> > >> > > >>>> Yea I really like that idea I'll see what I can do to update
> > the
> > >> kip
> > >> > > >> and
> > >> > > >>>> my pr when I have some time. I'm not sure how well creating
> the
> > >> > > >>>> kstreamaggregates will go though because at that point I will
> > >> have
> > >> > > >> thrown
> > >> > > >>>> away the type of the values. It will be type safe I just may
> > >> need to
> > >> > > >> do a
> > >> > > >>>> little forcing.
> > >> > > >>>>
> > >> > > >>>> Thanks,
> > >> > > >>>> Kyle
> > >> > > >>>>
> > >> > > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> >
> > >> > wrote:
> > >> > > >>>>
> > >> > > >>>>> Kyle,
> > >> > > >>>>>
> > >> > > >>>>> Thanks for the explanations, my previous read on the wiki
> > >> examples
> > >> > > was
> > >> > > >>>>> wrong.
> > >> > > >>>>>
> > >> > > >>>>> So I guess my motivation should be "reduced" to: can we move
> > the
> > >> > > >> window
> > >> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > >> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >> > > >>>>>
> > >> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > from
> > >> 3
> > >> > to
> > >> > > >> 2.
> > >> > > >>>>> 2. major: this is for extensibility of the APIs, and since
> we
> > >> are
> > >> > > >>> removing
> > >> > > >>>>> the "Evolving" annotations on Streams it may be harder to
> > >> change it
> > >> > > >>> again
> > >> > > >>>>> in the future. The extended use cases are that people wanted
> > to
> > >> > have
> > >> > > >>>>> windowed running aggregates on different granularities, e.g.
> > >> "give
> > >> > me
> > >> > > >>> the
> > >> > > >>>>> counts per-minute, per-hour, per-day and per-week", and
> today
> > in
> > >> > DSL
> > >> > > >> we
> > >> > > >>>>> need to specify that case in multiple aggregate operators,
> > which
> > >> > gets
> > >> > > >> a
> > >> > > >>>>> state store / changelog, etc. And it is possible to optimize
> > it
> > >> as
> > >> > > >> well
> > >> > > >>> to
> > >> > > >>>>> a single state store. Its implementation would be tricky as
> > you
> > >> > need
> > >> > > >> to
> > >> > > >>>>> contain different lengthed windows within your window store
> > but
> > >> > just
> > >> > > >>> from
> > >> > > >>>>> the public API point of view, it could be specified as:
> > >> > > >>>>>
> > >> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >> > > >>>>> "state-store-name");
> > >> > > >>>>>
> > >> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > >> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > >> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > >> > > >>>>>
> > >> > > >>>>> while underlying we are only using a single store
> > >> > "state-store-name"
> > >> > > >> for
> > >> > > >>>>> it.
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> Although this feature is out of the scope of this KIP, I'd
> > like
> > >> to
> > >> > > >>> discuss
> > >> > > >>>>> if we can "leave the door open" to make such changes without
> > >> > > modifying
> > >> > > >>> the
> > >> > > >>>>> public APIs .
> > >> > > >>>>>
> > >> > > >>>>> Guozhang
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > >> > > >>> winkelman.kyle@gmail.com
> > >> > > >>>>>>
> > >> > > >>>>> wrote:
> > >> > > >>>>>
> > >> > > >>>>>> I allow defining a single window/sessionwindow one time
> when
> > >> you
> > >> > > >> make
> > >> > > >>>>> the
> > >> > > >>>>>> cogroup call from a KGroupedStream. From then on you are
> > using
> > >> the
> > >> > > >>>>> cogroup
> > >> > > >>>>>> call from with in CogroupedKStream which doesnt accept any
> > >> > > >> additional
> > >> > > >>>>>> windows/sessionwindows.
> > >> > > >>>>>>
> > >> > > >>>>>> Is this what you meant by your question or did I
> > misunderstand?
> > >> > > >>>>>>
> > >> > > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> wangguoz@gmail.com
> > >
> > >> > > >> wrote:
> > >> > > >>>>>>
> > >> > > >>>>>> Another question that came to me is on "window alignment":
> > from
> > >> > the
> > >> > > >>> KIP
> > >> > > >>>>> it
> > >> > > >>>>>> seems you are allowing users to specify a (potentially
> > >> different)
> > >> > > >>> window
> > >> > > >>>>>> spec in each co-grouped input stream. So if these window
> > specs
> > >> are
> > >> > > >>>>>> different how should we "align" them with different input
> > >> > streams? I
> > >> > > >>>>> think
> > >> > > >>>>>> it is more natural to only specify on window spec in the
> > >> > > >>>>>>
> > >> > > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> And remove it from the cogroup() functions. WDYT?
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> Guozhang
> > >> > > >>>>>>
> > >> > > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > >> > wangguoz@gmail.com>
> > >> > > >>>>> wrote:
> > >> > > >>>>>>
> > >> > > >>>>>>> Thanks for the proposal Kyle, this is a quite common use
> > case
> > >> to
> > >> > > >>>>> support
> > >> > > >>>>>>> such multi-way table join (i.e. N source tables with N
> > >> aggregate
> > >> > > >>> func)
> > >> > > >>>>>> with
> > >> > > >>>>>>> a single store and N+1 serdes, I have seen lots of people
> > >> using
> > >> > > >> the
> > >> > > >>>>>>> low-level PAPI to achieve this goal.
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >> > > >>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>> wrote:
> > >> > > >>>>>>>
> > >> > > >>>>>>>> I like your point about not handling other cases such as
> > >> count
> > >> > > >> and
> > >> > > >>>>>> reduce.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> I think that reduce may not make sense because reduce
> > assumes
> > >> > > >> that
> > >> > > >>>>> the
> > >> > > >>>>>>>> input values are the same as the output values. With
> > cogroup
> > >> > > >> there
> > >> > > >>>>> may
> > >> > > >>>>>> be
> > >> > > >>>>>>>> multiple different input types and then your output type
> > >> cant be
> > >> > > >>>>>> multiple
> > >> > > >>>>>>>> different things. In the case where you have all matching
> > >> value
> > >> > > >>> types
> > >> > > >>>>>> you
> > >> > > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> As for count I think it is possible to call count on all
> > the
> > >> > > >>>>> individual
> > >> > > >>>>>>>> grouped streams and then do joins. Otherwise we could
> maybe
> > >> make
> > >> > > >> a
> > >> > > >>>>>> special
> > >> > > >>>>>>>> call in groupedstream for this case. Because in this case
> > we
> > >> > dont
> > >> > > >>>>> need
> > >> > > >>>>>> to
> > >> > > >>>>>>>> do type checking on the values. It could be similar to
> the
> > >> > > >> current
> > >> > > >>>>> count
> > >> > > >>>>>>>> methods but accept a var args of additonal grouped
> streams
> > as
> > >> > > >> well
> > >> > > >>>>> and
> > >> > > >>>>>>>> make
> > >> > > >>>>>>>> sure they have a key type of K.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> The way I have put the kip together is to ensure that we
> do
> > >> type
> > >> > > >>>>>> checking.
> > >> > > >>>>>>>> I don't see a way we could group them all first and then
> > >> make a
> > >> > > >>> call
> > >> > > >>>>> to
> > >> > > >>>>>>>> count, reduce, or aggregate because with aggregate they
> > would
> > >> > > >> need
> > >> > > >>> to
> > >> > > >>>>>> pass
> > >> > > >>>>>>>> a list of aggregators and we would have no way of type
> > >> checking
> > >> > > >>> that
> > >> > > >>>>>> they
> > >> > > >>>>>>>> match the grouped streams.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> Thanks,
> > >> > > >>>>>>>> Kyle
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
> > >> xavier@confluent.io>
> > >> > > >>>>> wrote:
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a
> > very
> > >> > > >>> useful
> > >> > > >>>>>>>>> addition and wanted to provide an additional use-case
> and
> > >> some
> > >> > > >>> more
> > >> > > >>>>>>>>> comments.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> This is actually a very common analytics use-case in the
> > >> > > >> ad-tech
> > >> > > >>>>>>>> industry.
> > >> > > >>>>>>>>> The typical setup will have an auction stream, an
> > impression
> > >> > > >>>>> stream,
> > >> > > >>>>>>>> and a
> > >> > > >>>>>>>>> click stream. Those three streams need to be combined to
> > >> > > >> compute
> > >> > > >>>>>>>> aggregate
> > >> > > >>>>>>>>> statistics (e.g. impression statistics, and
> click-through
> > >> > > >> rates),
> > >> > > >>>>>> since
> > >> > > >>>>>>>>> most of the attributes of interest are only present the
> > >> auction
> > >> > > >>>>>> stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> A simple way to do this is to co-group all the streams
> by
> > >> the
> > >> > > >>>>> auction
> > >> > > >>>>>>>> key,
> > >> > > >>>>>>>>> and process updates to the co-group as events for each
> > >> stream
> > >> > > >>> come
> > >> > > >>>>> in,
> > >> > > >>>>>>>>> keeping only one value from each stream before sending
> > >> > > >> downstream
> > >> > > >>>>> for
> > >> > > >>>>>>>>> further processing / aggregation.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> One could view the result of that co-group operation as
> a
> > >> > > >>> "KTable"
> > >> > > >>>>>> with
> > >> > > >>>>>>>>> multiple values per key. The key being the grouping key,
> > and
> > >> > > >> the
> > >> > > >>>>>> values
> > >> > > >>>>>>>>> consisting of one value per stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> What I like about Kyle's approach is that allows elegant
> > >> > > >>>>> co-grouping
> > >> > > >>>>>> of
> > >> > > >>>>>>>>> multiple streams without having to worry about the
> number
> > of
> > >> > > >>>>> streams,
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>> avoids dealing with Tuple types or other generic
> > interfaces
> > >> > > >> that
> > >> > > >>>>> could
> > >> > > >>>>>>>> get
> > >> > > >>>>>>>>> messy if we wanted to preserve all the value types in
> the
> > >> > > >>> resulting
> > >> > > >>>>>>>>> co-grouped stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> My only concern is that we only allow the cogroup +
> > >> aggregate
> > >> > > >>>>> combined
> > >> > > >>>>>>>>> operation. This forces the user to build their own tuple
> > >> > > >>>>> serialization
> > >> > > >>>>>>>>> format if they want to preserve the individual input
> > stream
> > >> > > >>> values
> > >> > > >>>>> as
> > >> > > >>>>>> a
> > >> > > >>>>>>>>> group. It also deviates quite a bit from our approach in
> > >> > > >>>>>> KGroupedStream
> > >> > > >>>>>>>>> which offers other operations, such as count and reduce,
> > >> which
> > >> > > >>>>> should
> > >> > > >>>>>>>> also
> > >> > > >>>>>>>>> be applicable to a co-grouped stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> Overall I still think this is a really useful addition,
> > but
> > >> I
> > >> > > >>> feel
> > >> > > >>>>> we
> > >> > > >>>>>>>>> haven't spend much time trying to explore alternative
> DSLs
> > >> that
> > >> > > >>>>> could
> > >> > > >>>>>>>> maybe
> > >> > > >>>>>>>>> generalize better or match our existing syntax more
> > closely.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> > >> > > >>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> wrote:
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>>> Eno, is there anyone else that is an expert in the
> kafka
> > >> > > >>> streams
> > >> > > >>>>>> realm
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>> I should reach out to for input?
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>> I believe Damian Guy is still planning on reviewing
> this
> > >> more
> > >> > > >>> in
> > >> > > >>>>>> depth
> > >> > > >>>>>>>>> so I
> > >> > > >>>>>>>>>> will wait for his inputs before continuing.
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> > >> > > >> eno.thereska@gmail.com
> > >> > > >>>>
> > >> > > >>>>>>>> wrote:
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> Thanks Kyle, good arguments.
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > >> > > >>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *- minor: could you add an exact example (similar to
> > what
> > >> > > >>>>> Jay’s
> > >> > > >>>>>>>>> example
> > >> > > >>>>>>>>>>> is,
> > >> > > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
> super
> > >> > > >>>>>> concrete?*
> > >> > > >>>>>>>>>>>> I have added a more concrete example to the KIP.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *- my main concern is that we’re exposing this
> > >> > > >> optimization
> > >> > > >>>>> to
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> DSL.
> > >> > > >>>>>>>>>>> In
> > >> > > >>>>>>>>>>>> an ideal world, an optimizer would take the existing
> > DSL
> > >> > > >>> and
> > >> > > >>>>> do
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>> right
> > >> > > >>>>>>>>>>>> thing under the covers (create just one state store,
> > >> > > >>> arrange
> > >> > > >>>>> the
> > >> > > >>>>>>>>> nodes
> > >> > > >>>>>>>>>>>> etc). The original DSL had a bunch of small,
> composable
> > >> > > >>>>> pieces
> > >> > > >>>>>>>>> (group,
> > >> > > >>>>>>>>>>>> aggregate, join) that this proposal groups together.
> > I’d
> > >> > > >>>>> like to
> > >> > > >>>>>>>> hear
> > >> > > >>>>>>>>>>> your
> > >> > > >>>>>>>>>>>> thoughts on whether it’s possible to do this
> > optimization
> > >> > > >>>>> with
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>>> current
> > >> > > >>>>>>>>>>>> DSL, at the topology builder level.*
> > >> > > >>>>>>>>>>>> You would have to make a lot of checks to understand
> if
> > >> > > >> it
> > >> > > >>> is
> > >> > > >>>>>> even
> > >> > > >>>>>>>>>>> possible
> > >> > > >>>>>>>>>>>> to make this optimization:
> > >> > > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> > >> > > >>>>>>>>>>>> 2. None of the intermediate KTables are used for
> > anything
> > >> > > >>>>> else.
> > >> > > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This
> may
> > be
> > >> > > >>>>>>>> impossible
> > >> > > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> > >> > > >>> topology
> > >> > > >>>>> has
> > >> > > >>>>>>>>>> already
> > >> > > >>>>>>>>>>>> been built.)
> > >> > > >>>>>>>>>>>> You would then need to make decisions during the
> > >> > > >>>>> optimization:
> > >> > > >>>>>>>>>>>> 1. Your new initializer would the composite of all
> the
> > >> > > >>>>>> individual
> > >> > > >>>>>>>>>>>> initializers and the valueJoiners.
> > >> > > >>>>>>>>>>>> 2. I am having a hard time thinking about how you
> would
> > >> > > >>> turn
> > >> > > >>>>> the
> > >> > > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> > >> > > >> would
> > >> > > >>>>> work
> > >> > > >>>>>> on
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>> final object, but this may be possible.
> > >> > > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> > >> > > >> would
> > >> > > >>>>> be
> > >> > > >>>>>> for
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>> aggregate values. None of the declared ones would be
> > >> > > >>>>> guaranteed
> > >> > > >>>>>> to
> > >> > > >>>>>>>>> hold
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>> final object. This would mean you must created a new
> > >> > > >> state
> > >> > > >>>>> store
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> not
> > >> > > >>>>>>>>>>>> created any of the declared ones.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> The main argument I have against it is even if it
> could
> > >> > > >> be
> > >> > > >>>>> done
> > >> > > >>>>>> I
> > >> > > >>>>>>>>> don't
> > >> > > >>>>>>>>>>>> know that we would want to have this be an
> optimization
> > >> > > >> in
> > >> > > >>>>> the
> > >> > > >>>>>>>>>> background
> > >> > > >>>>>>>>>>>> because the user would still be required to think
> about
> > >> > > >> all
> > >> > > >>>>> of
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> > >> > > >> about
> > >> > > >>>>> if
> > >> > > >>>>>>>> they
> > >> > > >>>>>>>>>> only
> > >> > > >>>>>>>>>>>> care about the final object.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> > >> > > >>> should
> > >> > > >>>>> be
> > >> > > >>>>>>>> part
> > >> > > >>>>>>>>> of
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> > >> > > >> want
> > >> > > >>> to
> > >> > > >>>>>>>> allow
> > >> > > >>>>>>>>>>> people
> > >> > > >>>>>>>>>>>> to join more than 2 or more streams in an easy way.
> > Right
> > >> > > >>>>> now I
> > >> > > >>>>>>>> don't
> > >> > > >>>>>>>>>>> think
> > >> > > >>>>>>>>>>>> we give them ways of handling this use case easily.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *-I think there will be scope for several such
> > >> > > >>> optimizations
> > >> > > >>>>> in
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> future
> > >> > > >>>>>>>>>>>> and perhaps at some point we need to think about
> > >> > > >> decoupling
> > >> > > >>>>> the
> > >> > > >>>>>>>> 1:1
> > >> > > >>>>>>>>>>> mapping
> > >> > > >>>>>>>>>>>> from the DSL into the physical topology.*
> > >> > > >>>>>>>>>>>> I would argue that cogroup is not just an
> optimization
> > it
> > >> > > >>> is
> > >> > > >>>>> a
> > >> > > >>>>>> new
> > >> > > >>>>>>>>> way
> > >> > > >>>>>>>>>>> for
> > >> > > >>>>>>>>>>>> the users to look at accomplishing a problem that
> > >> > > >> requires
> > >> > > >>>>>>>> multiple
> > >> > > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> > >> > > >> think
> > >> > > >>>>>> users
> > >> > > >>>>>>>>>> should
> > >> > > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal
> with
> > >> > > >>> their
> > >> > > >>>>>>>>>>> initializers,
> > >> > > >>>>>>>>>>>> serdes and stores if all they care about is the final
> > >> > > >>> object.
> > >> > > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> > >> > > >> supply
> > >> > > >>>>>>>> additional
> > >> > > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> > >> > > >> single
> > >> > > >>>>>>>> grouped
> > >> > > >>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>> making an aggregate call. This case is what I view an
> > >> > > >>>>>> optimization
> > >> > > >>>>>>>>> as,
> > >> > > >>>>>>>>>> we
> > >> > > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there
> was
> > >> > > >>> just
> > >> > > >>>>> a
> > >> > > >>>>>>>> call
> > >> > > >>>>>>>>> to
> > >> > > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> > >> > > >>>>>>>> KGroupedStream#cogroup.
> > >> > > >>>>>>>>> (I
> > >> > > >>>>>>>>>>>> would prefer to just write a warning saying that this
> > is
> > >> > > >>> not
> > >> > > >>>>> how
> > >> > > >>>>>>>>>> cogroup
> > >> > > >>>>>>>>>>> is
> > >> > > >>>>>>>>>>>> to be used.)
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>> Kyle
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> > >> > > >>>>>>>> eno.thereska@gmail.com
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> - minor: could you add an exact example (similar to
> > what
> > >> > > >>>>> Jay’s
> > >> > > >>>>>>>>> example
> > >> > > >>>>>>>>>>> is,
> > >> > > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
> > super
> > >> > > >>>>>> concrete?
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> - my main concern is that we’re exposing this
> > >> > > >> optimization
> > >> > > >>>>> to
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> DSL.
> > >> > > >>>>>>>>>>> In
> > >> > > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing
> > DSL
> > >> > > >>>>> and do
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> right
> > >> > > >>>>>>>>>>>>> thing under the covers (create just one state store,
> > >> > > >>> arrange
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> nodes
> > >> > > >>>>>>>>>>>>> etc). The original DSL had a bunch of small,
> > composable
> > >> > > >>>>> pieces
> > >> > > >>>>>>>>> (group,
> > >> > > >>>>>>>>>>>>> aggregate, join) that this proposal groups together.
> > I’d
> > >> > > >>>>> like
> > >> > > >>>>>> to
> > >> > > >>>>>>>>> hear
> > >> > > >>>>>>>>>>> your
> > >> > > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> > >> > > >> optimization
> > >> > > >>>>> with
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> current
> > >> > > >>>>>>>>>>>>> DSL, at the topology builder level.
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> I think there will be scope for several such
> > >> > > >> optimizations
> > >> > > >>>>> in
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>> future
> > >> > > >>>>>>>>>>>>> and perhaps at some point we need to think about
> > >> > > >>> decoupling
> > >> > > >>>>> the
> > >> > > >>>>>>>> 1:1
> > >> > > >>>>>>>>>>> mapping
> > >> > > >>>>>>>>>>>>> from the DSL into the physical topology.
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Thanks
> > >> > > >>>>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> > >> > > >> jay@confluent.io>
> > >> > > >>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> > >> > > >>> pretty
> > >> > > >>>>>>>> common.
> > >> > > >>>>>>>>> An
> > >> > > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> > >> > > >> customer
> > >> > > >>>>>>>> profile"
> > >> > > >>>>>>>>> use
> > >> > > >>>>>>>>>>>>> case
> > >> > > >>>>>>>>>>>>>> we often use. In that use case you have a dozen
> > systems
> > >> > > >>>>> each
> > >> > > >>>>>> of
> > >> > > >>>>>>>>> which
> > >> > > >>>>>>>>>>> has
> > >> > > >>>>>>>>>>>>>> some information about your customer (account
> > details,
> > >> > > >>>>>> settings,
> > >> > > >>>>>>>>>>> billing
> > >> > > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> > >> > > >> etc).
> > >> > > >>>>> Your
> > >> > > >>>>>>>> goal
> > >> > > >>>>>>>>> is
> > >> > > >>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>> join/munge these into a single profile record for
> > each
> > >> > > >>>>>> customer
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>>> has
> > >> > > >>>>>>>>>>>>>> all the relevant info in a usable form and is
> > >> > > >> up-to-date
> > >> > > >>>>> with
> > >> > > >>>>>>>> all
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>>>> source systems. If you implement that with kstreams
> > as
> > >> > > >> a
> > >> > > >>>>>>>> sequence
> > >> > > >>>>>>>>> of
> > >> > > >>>>>>>>>>>>> joins
> > >> > > >>>>>>>>>>>>>> i think today we'd fully materialize N-1
> intermediate
> > >> > > >>>>> tables.
> > >> > > >>>>>>>> But
> > >> > > >>>>>>>>>>> clearly
> > >> > > >>>>>>>>>>>>>> you only need a single stage to group all these
> > things
> > >> > > >>> that
> > >> > > >>>>>> are
> > >> > > >>>>>>>>>> already
> > >> > > >>>>>>>>>>>>>> co-partitioned. A distributed database would do
> this
> > >> > > >>> under
> > >> > > >>>>> the
> > >> > > >>>>>>>>> covers
> > >> > > >>>>>>>>>>>>> which
> > >> > > >>>>>>>>>>>>>> is arguably better (at least when it does the right
> > >> > > >>> thing)
> > >> > > >>>>> and
> > >> > > >>>>>>>>>> perhaps
> > >> > > >>>>>>>>>>> we
> > >> > > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know
> the
> > >> > > >>>>>>>> partitioning
> > >> > > >>>>>>>>> so
> > >> > > >>>>>>>>>> we
> > >> > > >>>>>>>>>>>>> may
> > >> > > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they
> > are
> > >> > > >>>>>> already
> > >> > > >>>>>>>>>>>>>> co-partitioned.
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> -Jay
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> > >> > > >>>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> > >> > > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> > >> > > >> other
> > >> > > >>>>>>>> platforms
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> spark and pig.
> > >> > > >>>>>>>>>>>>>>> https://spark.apache.org/docs/
> 0.6.2/api/core/spark/
> > >> > > >>>>>>>>>>>>> PairRDDFunctions.html
> > >> > > >>>>>>>>>>>>>>>
> > https://www.tutorialspoint.com/apache_pig/apache_pig_
> > >> > > >>>>>>>>>>>>> cogroup_operator.htm
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> > >> > > >>>>> damian.guy@gmail.com>
> > >> > > >>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> > >> > > >> outer
> > >> > > >>>>>> join?
> > >> > > >>>>>>>> So
> > >> > > >>>>>>>>> an
> > >> > > >>>>>>>>>>>>> input
> > >> > > >>>>>>>>>>>>>>>> on any stream will always produce a new
> aggregated
> > >> > > >>> value
> > >> > > >>>>> -
> > >> > > >>>>>> is
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>>>>>>> correct?
> > >> > > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> > >> > > >> current
> > >> > > >>>>>> value,
> > >> > > >>>>>>>>>>>>> aggregates
> > >> > > >>>>>>>>>>>>>>>> and forwards the result.
> > >> > > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit
> > more,
> > >> > > >>>>> but it
> > >> > > >>>>>>>>> seems
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> it
> > >> > > >>>>>>>>>>>>>>>> could be a useful optimization.
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> > >> > > >>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> I sure can. I have added the following
> description
> > >> > > >> to
> > >> > > >>> my
> > >> > > >>>>>>>> KIP. If
> > >> > > >>>>>>>>>>> this
> > >> > > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some
> more
> > >> > > >>> time
> > >> > > >>>>> to
> > >> > > >>>>>>>>> build a
> > >> > > >>>>>>>>>>>>>>>> diagram
> > >> > > >>>>>>>>>>>>>>>>> and make more of a step by step description:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Example with Current API:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic1").group
> ByKey().aggregate(
> > >> > > >>>>>> initializer1
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator1,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic2").group
> ByKey().aggregate(
> > >> > > >>>>>> initializer2
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator2,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic3").group
> ByKey().aggregate(
> > >> > > >>>>>> initializer3
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator3,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> table1.outerJoin(table2,
> > >> > > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> > >> > > >>>>> joinerOneTwoAndThree);
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores,
> > requires
> > >> > > >> 3
> > >> > > >>>>>>>>>> initializers,
> > >> > > >>>>>>>>>>>>>>> and 3
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to
> > user
> > >> > > >> to
> > >> > > >>>>>> define
> > >> > > >>>>>>>>> what
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2,
> V3).
> > >> > > >> They
> > >> > > >>>>> are
> > >> > > >>>>>>>> left
> > >> > > >>>>>>>>>>> with a
> > >> > > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all
> > the
> > >> > > >>>>> same
> > >> > > >>>>>> as
> > >> > > >>>>>>>> CG
> > >> > > >>>>>>>>>> and
> > >> > > >>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> two
> > >> > > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make
> them
> > >> > > >>>>>>>> intermediate
> > >> > > >>>>>>>>>>> states
> > >> > > >>>>>>>>>>>>>>>> such
> > >> > > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> > >> > > >> joiners
> > >> > > >>>>> use
> > >> > > >>>>>>>> those
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>> build
> > >> > > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something
> > the
> > >> > > >>> user
> > >> > > >>>>>>>> could
> > >> > > >>>>>>>>>> avoid
> > >> > > >>>>>>>>>>>>>>>>> thinking about with this KIP.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> > > >> will
> > >> > > >>>>> first
> > >> > > >>>>>>>> go
> > >> > > >>>>>>>>>>> through
> > >> > > >>>>>>>>>>>>>>> a
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate
> > from
> > >> > > >>>>>>>> storeName1.
> > >> > > >>>>>>>>>> It
> > >> > > >>>>>>>>>>>>>>> will
> > >> > > >>>>>>>>>>>>>>>>> produce this in the form of the first
> intermediate
> > >> > > >>> value
> > >> > > >>>>>> and
> > >> > > >>>>>>>> get
> > >> > > >>>>>>>>>>> sent
> > >> > > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will
> look
> > >> > > >> up
> > >> > > >>>>> the
> > >> > > >>>>>>>>> current
> > >> > > >>>>>>>>>>>>> value
> > >> > > >>>>>>>>>>>>>>>> of
> > >> > > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first
> > joiner
> > >> > > >> to
> > >> > > >>>>>>>> calculate
> > >> > > >>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> second
> > >> > > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> > >> > > >>> additional
> > >> > > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> > >> > > >>> current
> > >> > > >>>>>>>> value of
> > >> > > >>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>> key
> > >> > > >>>>>>>>>>>>>>>> in
> > >> > > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build
> the
> > >> > > >>> final
> > >> > > >>>>>>>>> aggregate
> > >> > > >>>>>>>>>>>>>>> value.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> If you think through all possibilities for
> > incoming
> > >> > > >>>>> topics
> > >> > > >>>>>>>> you
> > >> > > >>>>>>>>>> will
> > >> > > >>>>>>>>>>>>> see
> > >> > > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through
> all
> > >> > > >>> three
> > >> > > >>>>>>>> stores
> > >> > > >>>>>>>>>> are
> > >> > > >>>>>>>>>>>>>>>> queried
> > >> > > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this
> creates
> > N
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1)
> KTableKTableOuterJoins,
> > >> > > >> and
> > >> > > >>>>> N-1
> > >> > > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Example with Proposed API:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> > >> > > >>>>> builder.stream("topic1").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> > >> > > >>>>> builder.stream("topic2").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> > >> > > >>>>> builder.stream("topic3").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> > >> > > >>> grouped1.cogroup(initializer1,
> > >> > > >>>>>>>>>>> aggregator1,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> > >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> > >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> > >> > > >>>>>>>>>>>>>>>>>      .aggregate();
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore,
> > requires 1
> > >> > > >>>>>>>>> initializer,
> > >> > > >>>>>>>>>>> and
> > >> > > >>>>>>>>>>>>> 1
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry
> > about
> > >> > > >>> the
> > >> > > >>>>>>>>>>> intermediate
> > >> > > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think
> > about
> > >> > > >>> is
> > >> > > >>>>> how
> > >> > > >>>>>>>> each
> > >> > > >>>>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> > > >> will
> > >> > > >>>>> first
> > >> > > >>>>>>>> go
> > >> > > >>>>>>>>>>> through
> > >> > > >>>>>>>>>>>>>>> a
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate
> > from
> > >> > > >>>>>>>> storeName1.
> > >> > > >>>>>>>>>> It
> > >> > > >>>>>>>>>>>>>>> will
> > >> > > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update
> > the
> > >> > > >>>>> store
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> pass
> > >> > > >>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> new
> > >> > > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through
> the
> > >> > > >>>>>>>> KStreamCogroup
> > >> > > >>>>>>>>>>> which
> > >> > > >>>>>>>>>>>>>>> is
> > >> > > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and
> you
> > >> > > >> are
> > >> > > >>>>> done.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> > >> > > >> will
> > >> > > >>>>> only
> > >> > > >>>>>>>> every
> > >> > > >>>>>>>>>>>>>>> create N
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax
> <
> > >> > > >>>>>>>>>>>>> matthias@confluent.io
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> Kyle,
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little
> > slow,
> > >> > > >>>>> but I
> > >> > > >>>>>>>> could
> > >> > > >>>>>>>>>> not
> > >> > > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> > >> > > >>> concrete
> > >> > > >>>>>>>> example,
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> 3
> > >> > > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected
> > result),
> > >> > > >>> and
> > >> > > >>>>>> show
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>> difference between current way to to implement
> it
> > >> > > >> and
> > >> > > >>>>> the
> > >> > > >>>>>>>>>> proposed
> > >> > > >>>>>>>>>>>>>>> API?
> > >> > > >>>>>>>>>>>>>>>>>> This could also cover the internal processing
> to
> > >> > > >> see
> > >> > > >>>>> what
> > >> > > >>>>>>>> store
> > >> > > >>>>>>>>>>> calls
> > >> > > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you
> propose,
> > >> > > >> and
> > >> > > >>> it
> > >> > > >>>>>>>> would
> > >> > > >>>>>>>>>> help
> > >> > > >>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>> understand it better.
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> Thanks a lot!
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> -Matthias
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> > >> > > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found
> > here.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> > >> > > >> classes
> > >> > > >>>>> and
> > >> > > >>>>>> get
> > >> > > >>>>>>>>>> around
> > >> > > >>>>>>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> > >> > > >> additions.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> > >> > > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> > >> > > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> > >> > > >>>>>>>>> repartitionIfRequired
> > >> > > >>>>>>>>>>>>>>>> method.
> > >> > > >>>>>>>>>>>>>>>>> I
> > >> > > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more
> than
> > >> > > >> one
> > >> > > >>>>>>>> grouped
> > >> > > >>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> > >> > > >> name
> > >> > > >>>>> that
> > >> > > >>>>>>>>> someone
> > >> > > >>>>>>>>>>>>>>> can
> > >> > > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow
> > it
> > >> > > >> to
> > >> > > >>>>> fall
> > >> > > >>>>>>>> back
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped
> > tables?
> > >> > > >>> This
> > >> > > >>>>>>>> would
> > >> > > >>>>>>>>> be
> > >> > > >>>>>>>>>>>>>>>> pretty
> > >> > > >>>>>>>>>>>>>>>>>> easy
> > >> > > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing
> session
> > >> > > >>> stores
> > >> > > >>>>>> and
> > >> > > >>>>>>>>>>> windowed
> > >> > > >>>>>>>>>>>>>>>>>> stores
> > >> > > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate
> and
> > >> > > >>>>>>>>>>>>>>> KTableWindowAggregate
> > >> > > >>>>>>>>>>>>>>>>>>> implementations.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>> Kyle
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> > >> > > >>>>>>>>> eno.thereska@gmail.com>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> > >> > > >>>>>>>>> damian.guy@gmail.com>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i
> haven't
> > >> > > >> had
> > >> > > >>>>> the
> > >> > > >>>>>>>>> chance
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>> look
> > >> > > >>>>>>>>>>>>>>>>>> at
> > >> > > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to
> > look
> > >> > > >>>>> into
> > >> > > >>>>>> it
> > >> > > >>>>>>>>>>>>>>> tomorrow.
> > >> > > >>>>>>>>>>>>>>>>> For
> > >> > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against
> > kafka
> > >> > > >>>>> trunk
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> mark
> > >> > > >>>>>>>>>>>>>>> it
> > >> > > >>>>>>>>>>>>>>>> as
> > >> > > >>>>>>>>>>>>>>>>>>>> WIP?
> > >> > > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have
> > done.
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>>>> Damian
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> > >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> > >> > > >> some
> > >> > > >>>>>>>> attention
> > >> > > >>>>>>>>> to
> > >> > > >>>>>>>>>>> my
> > >> > > >>>>>>>>>>>>>>>> KIP
> > >> > > >>>>>>>>>>>>>>>>>> as
> > >> > > >>>>>>>>>>>>>>>>>>>> I
> > >> > > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> > >> > > >> This
> > >> > > >>>>> is my
> > >> > > >>>>>>>>> first
> > >> > > >>>>>>>>>>> KIP
> > >> > > >>>>>>>>>>>>>>>> and
> > >> > > >>>>>>>>>>>>>>>>>> my
> > >> > > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so
> > I'm
> > >> > > >>>>> sure I
> > >> > > >>>>>>>> did
> > >> > > >>>>>>>>>>>>>>>> something
> > >> > > >>>>>>>>>>>>>>>>>>>> wrong.
> > >> > > >>>>>>>>>>>>>>>>>>>>>> ;)
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> > >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> > >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> > >> > > >> discussion
> > >> > > >>>>> about
> > >> > > >>>>>>>>> adding
> > >> > > >>>>>>>>>>>>>>>>> cogroup
> > >> > > >>>>>>>>>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > >> > > >>>>>> confluence/display/KAFKA/KIP-
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation
> here:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> --
> > >> > > >>>>>>> -- Guozhang
> > >> > > >>>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> --
> > >> > > >>>>>> -- Guozhang
> > >> > > >>>>>>
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> --
> > >> > > >>>>> -- Guozhang
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>
> > >> > > >>
> > >> > >
> > >> > >
> > >> >
> > >> >
> > >> > --
> > >> > -- Guozhang
> > >> >
> > >>
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
I chose the current way so if you make multiple tables you don't need to
supply the serde and initializer multiple times. It is true that you
wouldnt need the serde if you use a statestoresupplier but I think we could
note that in the method call.

I am fine with the first option if thats what people like. Maybe Eno or
Damian can give their opinion.

I dont really like the kstreambuilder option cause I think it is kind of
hard to find unless you know it's there.

On Jun 8, 2017 1:51 PM, "Xavier Léauté" <xa...@confluent.io> wrote:

> Another reason for the serde not to be in the first cogroup call, is that
> the serde should not be required if you pass a StateStoreSupplier to
> aggregate()
>
> Regarding the aggregated type <T> I don't the why initializer should be
> favored over aggregator to define the type. In my mind separating the
> initializer into the last aggregate call clearly indicates that the
> initializer is independent of any of the aggregators or streams and that we
> don't wait for grouped1 events to initialize the co-group.
>
> On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > On a second thought... This is the current proposal API
> >
> >
> > ```
> >
> > <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer,
> final
> > Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> > aggValueSerde)
> >
> > ```
> >
> >
> > If we do not have the initializer in the first co-group it might be a bit
> > awkward for users to specify the aggregator that returns a typed <T>
> value?
> > Maybe it is still better to put these two functions in the same api?
> >
> >
> >
> > Guozhang
> >
> > On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > This suggestion lgtm. I would vote for the first alternative than
> adding
> > > it to the `KStreamBuilder` though.
> > >
> > > On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
> > > wrote:
> > >
> > >> I have a minor suggestion to make the API a little bit more symmetric.
> > >> I feel it would make more sense to move the initializer and serde to
> the
> > >> final aggregate statement, since the serde only applies to the state
> > >> store,
> > >> and the initializer doesn't bear any relation to the first group in
> > >> particular. It would end up looking like this:
> > >>
> > >> KTable<K, CG> cogrouped =
> > >>     grouped1.cogroup(aggregator1)
> > >>             .cogroup(grouped2, aggregator2)
> > >>             .cogroup(grouped3, aggregator3)
> > >>             .aggregate(initializer1, aggValueSerde, storeName1);
> > >>
> > >> Alternatively, we could move the the first cogroup() method to
> > >> KStreamBuilder, similar to how we have .merge()
> > >> and end up with an api that would be even more symmetric.
> > >>
> > >> KStreamBuilder.cogroup(grouped1, aggregator1)
> > >>               .cogroup(grouped2, aggregator2)
> > >>               .cogroup(grouped3, aggregator3)
> > >>               .aggregate(initializer1, aggValueSerde, storeName1);
> > >>
> > >> This doesn't have to be a blocker, but I thought it would make the API
> > >> just
> > >> a tad cleaner.
> > >>
> > >> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >>
> > >> > Kyle,
> > >> >
> > >> > Thanks a lot for the updated KIP. It looks good to me.
> > >> >
> > >> >
> > >> > Guozhang
> > >> >
> > >> >
> > >> > On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> > wrote:
> > >> >
> > >> > > This makes much more sense to me. +1
> > >> > >
> > >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >> winkelman.kyle@gmail.com>
> > >> > > wrote:
> > >> > > >
> > >> > > > I have updated the KIP and my PR. Let me know what you think.
> > >> > > > To created a cogrouped stream just call cogroup on a
> > KgroupedStream
> > >> and
> > >> > > > supply the initializer, aggValueSerde, and an aggregator. Then
> > >> continue
> > >> > > > adding kgroupedstreams and aggregators. Then call one of the
> many
> > >> > > aggregate
> > >> > > > calls to create a KTable.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Kyle
> > >> > > >
> > >> > > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com>
> > wrote:
> > >> > > >
> > >> > > >> Hi Kyle,
> > >> > > >>
> > >> > > >> Thanks for the update. I think just one initializer makes sense
> > as
> > >> it
> > >> > > >> should only be called once per key and generally it is just
> going
> > >> to
> > >> > > create
> > >> > > >> a new instance of whatever the Aggregate class is.
> > >> > > >>
> > >> > > >> Cheers,
> > >> > > >> Damian
> > >> > > >>
> > >> > > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> > >> winkelman.kyle@gmail.com
> > >> > >
> > >> > > >> wrote:
> > >> > > >>
> > >> > > >>> Hello all,
> > >> > > >>>
> > >> > > >>> I have spent some more time on this and the best alternative I
> > >> have
> > >> > > come
> > >> > > >> up
> > >> > > >>> with is:
> > >> > > >>> KGroupedStream has a single cogroup call that takes an
> > initializer
> > >> > and
> > >> > > an
> > >> > > >>> aggregator.
> > >> > > >>> CogroupedKStream has a cogroup call that takes additional
> > >> > groupedStream
> > >> > > >>> aggregator pairs.
> > >> > > >>> CogroupedKStream has multiple aggregate methods that create
> the
> > >> > > different
> > >> > > >>> stores.
> > >> > > >>>
> > >> > > >>> I plan on updating the kip but I want people's input on if we
> > >> should
> > >> > > have
> > >> > > >>> the initializer be passed in once at the beginning or if we
> > should
> > >> > > >> instead
> > >> > > >>> have the initializer be required for each call to one of the
> > >> > aggregate
> > >> > > >>> calls. The first makes more sense to me but doesnt allow the
> > user
> > >> to
> > >> > > >>> specify different initializers for different tables.
> > >> > > >>>
> > >> > > >>> Thanks,
> > >> > > >>> Kyle
> > >> > > >>>
> > >> > > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> > >> winkelman.kyle@gmail.com>
> > >> > > >>> wrote:
> > >> > > >>>
> > >> > > >>>> Yea I really like that idea I'll see what I can do to update
> > the
> > >> kip
> > >> > > >> and
> > >> > > >>>> my pr when I have some time. I'm not sure how well creating
> the
> > >> > > >>>> kstreamaggregates will go though because at that point I will
> > >> have
> > >> > > >> thrown
> > >> > > >>>> away the type of the values. It will be type safe I just may
> > >> need to
> > >> > > >> do a
> > >> > > >>>> little forcing.
> > >> > > >>>>
> > >> > > >>>> Thanks,
> > >> > > >>>> Kyle
> > >> > > >>>>
> > >> > > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wangguoz@gmail.com
> >
> > >> > wrote:
> > >> > > >>>>
> > >> > > >>>>> Kyle,
> > >> > > >>>>>
> > >> > > >>>>> Thanks for the explanations, my previous read on the wiki
> > >> examples
> > >> > > was
> > >> > > >>>>> wrong.
> > >> > > >>>>>
> > >> > > >>>>> So I guess my motivation should be "reduced" to: can we move
> > the
> > >> > > >> window
> > >> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > >> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >> > > >>>>>
> > >> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> > from
> > >> 3
> > >> > to
> > >> > > >> 2.
> > >> > > >>>>> 2. major: this is for extensibility of the APIs, and since
> we
> > >> are
> > >> > > >>> removing
> > >> > > >>>>> the "Evolving" annotations on Streams it may be harder to
> > >> change it
> > >> > > >>> again
> > >> > > >>>>> in the future. The extended use cases are that people wanted
> > to
> > >> > have
> > >> > > >>>>> windowed running aggregates on different granularities, e.g.
> > >> "give
> > >> > me
> > >> > > >>> the
> > >> > > >>>>> counts per-minute, per-hour, per-day and per-week", and
> today
> > in
> > >> > DSL
> > >> > > >> we
> > >> > > >>>>> need to specify that case in multiple aggregate operators,
> > which
> > >> > gets
> > >> > > >> a
> > >> > > >>>>> state store / changelog, etc. And it is possible to optimize
> > it
> > >> as
> > >> > > >> well
> > >> > > >>> to
> > >> > > >>>>> a single state store. Its implementation would be tricky as
> > you
> > >> > need
> > >> > > >> to
> > >> > > >>>>> contain different lengthed windows within your window store
> > but
> > >> > just
> > >> > > >>> from
> > >> > > >>>>> the public API point of view, it could be specified as:
> > >> > > >>>>>
> > >> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >> > > >>>>> "state-store-name");
> > >> > > >>>>>
> > >> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > >> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > >> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > >> > > >>>>>
> > >> > > >>>>> while underlying we are only using a single store
> > >> > "state-store-name"
> > >> > > >> for
> > >> > > >>>>> it.
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> Although this feature is out of the scope of this KIP, I'd
> > like
> > >> to
> > >> > > >>> discuss
> > >> > > >>>>> if we can "leave the door open" to make such changes without
> > >> > > modifying
> > >> > > >>> the
> > >> > > >>>>> public APIs .
> > >> > > >>>>>
> > >> > > >>>>> Guozhang
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > >> > > >>> winkelman.kyle@gmail.com
> > >> > > >>>>>>
> > >> > > >>>>> wrote:
> > >> > > >>>>>
> > >> > > >>>>>> I allow defining a single window/sessionwindow one time
> when
> > >> you
> > >> > > >> make
> > >> > > >>>>> the
> > >> > > >>>>>> cogroup call from a KGroupedStream. From then on you are
> > using
> > >> the
> > >> > > >>>>> cogroup
> > >> > > >>>>>> call from with in CogroupedKStream which doesnt accept any
> > >> > > >> additional
> > >> > > >>>>>> windows/sessionwindows.
> > >> > > >>>>>>
> > >> > > >>>>>> Is this what you meant by your question or did I
> > misunderstand?
> > >> > > >>>>>>
> > >> > > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <
> wangguoz@gmail.com
> > >
> > >> > > >> wrote:
> > >> > > >>>>>>
> > >> > > >>>>>> Another question that came to me is on "window alignment":
> > from
> > >> > the
> > >> > > >>> KIP
> > >> > > >>>>> it
> > >> > > >>>>>> seems you are allowing users to specify a (potentially
> > >> different)
> > >> > > >>> window
> > >> > > >>>>>> spec in each co-grouped input stream. So if these window
> > specs
> > >> are
> > >> > > >>>>>> different how should we "align" them with different input
> > >> > streams? I
> > >> > > >>>>> think
> > >> > > >>>>>> it is more natural to only specify on window spec in the
> > >> > > >>>>>>
> > >> > > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> And remove it from the cogroup() functions. WDYT?
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> Guozhang
> > >> > > >>>>>>
> > >> > > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > >> > wangguoz@gmail.com>
> > >> > > >>>>> wrote:
> > >> > > >>>>>>
> > >> > > >>>>>>> Thanks for the proposal Kyle, this is a quite common use
> > case
> > >> to
> > >> > > >>>>> support
> > >> > > >>>>>>> such multi-way table join (i.e. N source tables with N
> > >> aggregate
> > >> > > >>> func)
> > >> > > >>>>>> with
> > >> > > >>>>>>> a single store and N+1 serdes, I have seen lots of people
> > >> using
> > >> > > >> the
> > >> > > >>>>>>> low-level PAPI to achieve this goal.
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >> > > >>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>> wrote:
> > >> > > >>>>>>>
> > >> > > >>>>>>>> I like your point about not handling other cases such as
> > >> count
> > >> > > >> and
> > >> > > >>>>>> reduce.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> I think that reduce may not make sense because reduce
> > assumes
> > >> > > >> that
> > >> > > >>>>> the
> > >> > > >>>>>>>> input values are the same as the output values. With
> > cogroup
> > >> > > >> there
> > >> > > >>>>> may
> > >> > > >>>>>> be
> > >> > > >>>>>>>> multiple different input types and then your output type
> > >> cant be
> > >> > > >>>>>> multiple
> > >> > > >>>>>>>> different things. In the case where you have all matching
> > >> value
> > >> > > >>> types
> > >> > > >>>>>> you
> > >> > > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> As for count I think it is possible to call count on all
> > the
> > >> > > >>>>> individual
> > >> > > >>>>>>>> grouped streams and then do joins. Otherwise we could
> maybe
> > >> make
> > >> > > >> a
> > >> > > >>>>>> special
> > >> > > >>>>>>>> call in groupedstream for this case. Because in this case
> > we
> > >> > dont
> > >> > > >>>>> need
> > >> > > >>>>>> to
> > >> > > >>>>>>>> do type checking on the values. It could be similar to
> the
> > >> > > >> current
> > >> > > >>>>> count
> > >> > > >>>>>>>> methods but accept a var args of additonal grouped
> streams
> > as
> > >> > > >> well
> > >> > > >>>>> and
> > >> > > >>>>>>>> make
> > >> > > >>>>>>>> sure they have a key type of K.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> The way I have put the kip together is to ensure that we
> do
> > >> type
> > >> > > >>>>>> checking.
> > >> > > >>>>>>>> I don't see a way we could group them all first and then
> > >> make a
> > >> > > >>> call
> > >> > > >>>>> to
> > >> > > >>>>>>>> count, reduce, or aggregate because with aggregate they
> > would
> > >> > > >> need
> > >> > > >>> to
> > >> > > >>>>>> pass
> > >> > > >>>>>>>> a list of aggregators and we would have no way of type
> > >> checking
> > >> > > >>> that
> > >> > > >>>>>> they
> > >> > > >>>>>>>> match the grouped streams.
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> Thanks,
> > >> > > >>>>>>>> Kyle
> > >> > > >>>>>>>>
> > >> > > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
> > >> xavier@confluent.io>
> > >> > > >>>>> wrote:
> > >> > > >>>>>>>>
> > >> > > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a
> > very
> > >> > > >>> useful
> > >> > > >>>>>>>>> addition and wanted to provide an additional use-case
> and
> > >> some
> > >> > > >>> more
> > >> > > >>>>>>>>> comments.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> This is actually a very common analytics use-case in the
> > >> > > >> ad-tech
> > >> > > >>>>>>>> industry.
> > >> > > >>>>>>>>> The typical setup will have an auction stream, an
> > impression
> > >> > > >>>>> stream,
> > >> > > >>>>>>>> and a
> > >> > > >>>>>>>>> click stream. Those three streams need to be combined to
> > >> > > >> compute
> > >> > > >>>>>>>> aggregate
> > >> > > >>>>>>>>> statistics (e.g. impression statistics, and
> click-through
> > >> > > >> rates),
> > >> > > >>>>>> since
> > >> > > >>>>>>>>> most of the attributes of interest are only present the
> > >> auction
> > >> > > >>>>>> stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> A simple way to do this is to co-group all the streams
> by
> > >> the
> > >> > > >>>>> auction
> > >> > > >>>>>>>> key,
> > >> > > >>>>>>>>> and process updates to the co-group as events for each
> > >> stream
> > >> > > >>> come
> > >> > > >>>>> in,
> > >> > > >>>>>>>>> keeping only one value from each stream before sending
> > >> > > >> downstream
> > >> > > >>>>> for
> > >> > > >>>>>>>>> further processing / aggregation.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> One could view the result of that co-group operation as
> a
> > >> > > >>> "KTable"
> > >> > > >>>>>> with
> > >> > > >>>>>>>>> multiple values per key. The key being the grouping key,
> > and
> > >> > > >> the
> > >> > > >>>>>> values
> > >> > > >>>>>>>>> consisting of one value per stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> What I like about Kyle's approach is that allows elegant
> > >> > > >>>>> co-grouping
> > >> > > >>>>>> of
> > >> > > >>>>>>>>> multiple streams without having to worry about the
> number
> > of
> > >> > > >>>>> streams,
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>> avoids dealing with Tuple types or other generic
> > interfaces
> > >> > > >> that
> > >> > > >>>>> could
> > >> > > >>>>>>>> get
> > >> > > >>>>>>>>> messy if we wanted to preserve all the value types in
> the
> > >> > > >>> resulting
> > >> > > >>>>>>>>> co-grouped stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> My only concern is that we only allow the cogroup +
> > >> aggregate
> > >> > > >>>>> combined
> > >> > > >>>>>>>>> operation. This forces the user to build their own tuple
> > >> > > >>>>> serialization
> > >> > > >>>>>>>>> format if they want to preserve the individual input
> > stream
> > >> > > >>> values
> > >> > > >>>>> as
> > >> > > >>>>>> a
> > >> > > >>>>>>>>> group. It also deviates quite a bit from our approach in
> > >> > > >>>>>> KGroupedStream
> > >> > > >>>>>>>>> which offers other operations, such as count and reduce,
> > >> which
> > >> > > >>>>> should
> > >> > > >>>>>>>> also
> > >> > > >>>>>>>>> be applicable to a co-grouped stream.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> Overall I still think this is a really useful addition,
> > but
> > >> I
> > >> > > >>> feel
> > >> > > >>>>> we
> > >> > > >>>>>>>>> haven't spend much time trying to explore alternative
> DSLs
> > >> that
> > >> > > >>>>> could
> > >> > > >>>>>>>> maybe
> > >> > > >>>>>>>>> generalize better or match our existing syntax more
> > closely.
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> > >> > > >>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>> wrote:
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>>> Eno, is there anyone else that is an expert in the
> kafka
> > >> > > >>> streams
> > >> > > >>>>>> realm
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>> I should reach out to for input?
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>> I believe Damian Guy is still planning on reviewing
> this
> > >> more
> > >> > > >>> in
> > >> > > >>>>>> depth
> > >> > > >>>>>>>>> so I
> > >> > > >>>>>>>>>> will wait for his inputs before continuing.
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> > >> > > >> eno.thereska@gmail.com
> > >> > > >>>>
> > >> > > >>>>>>>> wrote:
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> Thanks Kyle, good arguments.
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > >> > > >>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *- minor: could you add an exact example (similar to
> > what
> > >> > > >>>>> Jay’s
> > >> > > >>>>>>>>> example
> > >> > > >>>>>>>>>>> is,
> > >> > > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
> super
> > >> > > >>>>>> concrete?*
> > >> > > >>>>>>>>>>>> I have added a more concrete example to the KIP.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *- my main concern is that we’re exposing this
> > >> > > >> optimization
> > >> > > >>>>> to
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> DSL.
> > >> > > >>>>>>>>>>> In
> > >> > > >>>>>>>>>>>> an ideal world, an optimizer would take the existing
> > DSL
> > >> > > >>> and
> > >> > > >>>>> do
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>> right
> > >> > > >>>>>>>>>>>> thing under the covers (create just one state store,
> > >> > > >>> arrange
> > >> > > >>>>> the
> > >> > > >>>>>>>>> nodes
> > >> > > >>>>>>>>>>>> etc). The original DSL had a bunch of small,
> composable
> > >> > > >>>>> pieces
> > >> > > >>>>>>>>> (group,
> > >> > > >>>>>>>>>>>> aggregate, join) that this proposal groups together.
> > I’d
> > >> > > >>>>> like to
> > >> > > >>>>>>>> hear
> > >> > > >>>>>>>>>>> your
> > >> > > >>>>>>>>>>>> thoughts on whether it’s possible to do this
> > optimization
> > >> > > >>>>> with
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>>> current
> > >> > > >>>>>>>>>>>> DSL, at the topology builder level.*
> > >> > > >>>>>>>>>>>> You would have to make a lot of checks to understand
> if
> > >> > > >> it
> > >> > > >>> is
> > >> > > >>>>>> even
> > >> > > >>>>>>>>>>> possible
> > >> > > >>>>>>>>>>>> to make this optimization:
> > >> > > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> > >> > > >>>>>>>>>>>> 2. None of the intermediate KTables are used for
> > anything
> > >> > > >>>>> else.
> > >> > > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This
> may
> > be
> > >> > > >>>>>>>> impossible
> > >> > > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> > >> > > >>> topology
> > >> > > >>>>> has
> > >> > > >>>>>>>>>> already
> > >> > > >>>>>>>>>>>> been built.)
> > >> > > >>>>>>>>>>>> You would then need to make decisions during the
> > >> > > >>>>> optimization:
> > >> > > >>>>>>>>>>>> 1. Your new initializer would the composite of all
> the
> > >> > > >>>>>> individual
> > >> > > >>>>>>>>>>>> initializers and the valueJoiners.
> > >> > > >>>>>>>>>>>> 2. I am having a hard time thinking about how you
> would
> > >> > > >>> turn
> > >> > > >>>>> the
> > >> > > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> > >> > > >> would
> > >> > > >>>>> work
> > >> > > >>>>>> on
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>> final object, but this may be possible.
> > >> > > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> > >> > > >> would
> > >> > > >>>>> be
> > >> > > >>>>>> for
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>> aggregate values. None of the declared ones would be
> > >> > > >>>>> guaranteed
> > >> > > >>>>>> to
> > >> > > >>>>>>>>> hold
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>> final object. This would mean you must created a new
> > >> > > >> state
> > >> > > >>>>> store
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> not
> > >> > > >>>>>>>>>>>> created any of the declared ones.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> The main argument I have against it is even if it
> could
> > >> > > >> be
> > >> > > >>>>> done
> > >> > > >>>>>> I
> > >> > > >>>>>>>>> don't
> > >> > > >>>>>>>>>>>> know that we would want to have this be an
> optimization
> > >> > > >> in
> > >> > > >>>>> the
> > >> > > >>>>>>>>>> background
> > >> > > >>>>>>>>>>>> because the user would still be required to think
> about
> > >> > > >> all
> > >> > > >>>>> of
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> > >> > > >> about
> > >> > > >>>>> if
> > >> > > >>>>>>>> they
> > >> > > >>>>>>>>>> only
> > >> > > >>>>>>>>>>>> care about the final object.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> > >> > > >>> should
> > >> > > >>>>> be
> > >> > > >>>>>>>> part
> > >> > > >>>>>>>>> of
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> > >> > > >> want
> > >> > > >>> to
> > >> > > >>>>>>>> allow
> > >> > > >>>>>>>>>>> people
> > >> > > >>>>>>>>>>>> to join more than 2 or more streams in an easy way.
> > Right
> > >> > > >>>>> now I
> > >> > > >>>>>>>> don't
> > >> > > >>>>>>>>>>> think
> > >> > > >>>>>>>>>>>> we give them ways of handling this use case easily.
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> *-I think there will be scope for several such
> > >> > > >>> optimizations
> > >> > > >>>>> in
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> future
> > >> > > >>>>>>>>>>>> and perhaps at some point we need to think about
> > >> > > >> decoupling
> > >> > > >>>>> the
> > >> > > >>>>>>>> 1:1
> > >> > > >>>>>>>>>>> mapping
> > >> > > >>>>>>>>>>>> from the DSL into the physical topology.*
> > >> > > >>>>>>>>>>>> I would argue that cogroup is not just an
> optimization
> > it
> > >> > > >>> is
> > >> > > >>>>> a
> > >> > > >>>>>> new
> > >> > > >>>>>>>>> way
> > >> > > >>>>>>>>>>> for
> > >> > > >>>>>>>>>>>> the users to look at accomplishing a problem that
> > >> > > >> requires
> > >> > > >>>>>>>> multiple
> > >> > > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> > >> > > >> think
> > >> > > >>>>>> users
> > >> > > >>>>>>>>>> should
> > >> > > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal
> with
> > >> > > >>> their
> > >> > > >>>>>>>>>>> initializers,
> > >> > > >>>>>>>>>>>> serdes and stores if all they care about is the final
> > >> > > >>> object.
> > >> > > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> > >> > > >> supply
> > >> > > >>>>>>>> additional
> > >> > > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> > >> > > >> single
> > >> > > >>>>>>>> grouped
> > >> > > >>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>> making an aggregate call. This case is what I view an
> > >> > > >>>>>> optimization
> > >> > > >>>>>>>>> as,
> > >> > > >>>>>>>>>> we
> > >> > > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there
> was
> > >> > > >>> just
> > >> > > >>>>> a
> > >> > > >>>>>>>> call
> > >> > > >>>>>>>>> to
> > >> > > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> > >> > > >>>>>>>> KGroupedStream#cogroup.
> > >> > > >>>>>>>>> (I
> > >> > > >>>>>>>>>>>> would prefer to just write a warning saying that this
> > is
> > >> > > >>> not
> > >> > > >>>>> how
> > >> > > >>>>>>>>>> cogroup
> > >> > > >>>>>>>>>>> is
> > >> > > >>>>>>>>>>>> to be used.)
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>> Kyle
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> > >> > > >>>>>>>> eno.thereska@gmail.com
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> - minor: could you add an exact example (similar to
> > what
> > >> > > >>>>> Jay’s
> > >> > > >>>>>>>>> example
> > >> > > >>>>>>>>>>> is,
> > >> > > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
> > super
> > >> > > >>>>>> concrete?
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> - my main concern is that we’re exposing this
> > >> > > >> optimization
> > >> > > >>>>> to
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> DSL.
> > >> > > >>>>>>>>>>> In
> > >> > > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing
> > DSL
> > >> > > >>>>> and do
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> right
> > >> > > >>>>>>>>>>>>> thing under the covers (create just one state store,
> > >> > > >>> arrange
> > >> > > >>>>>> the
> > >> > > >>>>>>>>> nodes
> > >> > > >>>>>>>>>>>>> etc). The original DSL had a bunch of small,
> > composable
> > >> > > >>>>> pieces
> > >> > > >>>>>>>>> (group,
> > >> > > >>>>>>>>>>>>> aggregate, join) that this proposal groups together.
> > I’d
> > >> > > >>>>> like
> > >> > > >>>>>> to
> > >> > > >>>>>>>>> hear
> > >> > > >>>>>>>>>>> your
> > >> > > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> > >> > > >> optimization
> > >> > > >>>>> with
> > >> > > >>>>>>>> the
> > >> > > >>>>>>>>>>> current
> > >> > > >>>>>>>>>>>>> DSL, at the topology builder level.
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> I think there will be scope for several such
> > >> > > >> optimizations
> > >> > > >>>>> in
> > >> > > >>>>>> the
> > >> > > >>>>>>>>>> future
> > >> > > >>>>>>>>>>>>> and perhaps at some point we need to think about
> > >> > > >>> decoupling
> > >> > > >>>>> the
> > >> > > >>>>>>>> 1:1
> > >> > > >>>>>>>>>>> mapping
> > >> > > >>>>>>>>>>>>> from the DSL into the physical topology.
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>> Thanks
> > >> > > >>>>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> > >> > > >> jay@confluent.io>
> > >> > > >>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> > >> > > >>> pretty
> > >> > > >>>>>>>> common.
> > >> > > >>>>>>>>> An
> > >> > > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> > >> > > >> customer
> > >> > > >>>>>>>> profile"
> > >> > > >>>>>>>>> use
> > >> > > >>>>>>>>>>>>> case
> > >> > > >>>>>>>>>>>>>> we often use. In that use case you have a dozen
> > systems
> > >> > > >>>>> each
> > >> > > >>>>>> of
> > >> > > >>>>>>>>> which
> > >> > > >>>>>>>>>>> has
> > >> > > >>>>>>>>>>>>>> some information about your customer (account
> > details,
> > >> > > >>>>>> settings,
> > >> > > >>>>>>>>>>> billing
> > >> > > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> > >> > > >> etc).
> > >> > > >>>>> Your
> > >> > > >>>>>>>> goal
> > >> > > >>>>>>>>> is
> > >> > > >>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>> join/munge these into a single profile record for
> > each
> > >> > > >>>>>> customer
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>>> has
> > >> > > >>>>>>>>>>>>>> all the relevant info in a usable form and is
> > >> > > >> up-to-date
> > >> > > >>>>> with
> > >> > > >>>>>>>> all
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>>>> source systems. If you implement that with kstreams
> > as
> > >> > > >> a
> > >> > > >>>>>>>> sequence
> > >> > > >>>>>>>>> of
> > >> > > >>>>>>>>>>>>> joins
> > >> > > >>>>>>>>>>>>>> i think today we'd fully materialize N-1
> intermediate
> > >> > > >>>>> tables.
> > >> > > >>>>>>>> But
> > >> > > >>>>>>>>>>> clearly
> > >> > > >>>>>>>>>>>>>> you only need a single stage to group all these
> > things
> > >> > > >>> that
> > >> > > >>>>>> are
> > >> > > >>>>>>>>>> already
> > >> > > >>>>>>>>>>>>>> co-partitioned. A distributed database would do
> this
> > >> > > >>> under
> > >> > > >>>>> the
> > >> > > >>>>>>>>> covers
> > >> > > >>>>>>>>>>>>> which
> > >> > > >>>>>>>>>>>>>> is arguably better (at least when it does the right
> > >> > > >>> thing)
> > >> > > >>>>> and
> > >> > > >>>>>>>>>> perhaps
> > >> > > >>>>>>>>>>> we
> > >> > > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know
> the
> > >> > > >>>>>>>> partitioning
> > >> > > >>>>>>>>> so
> > >> > > >>>>>>>>>> we
> > >> > > >>>>>>>>>>>>> may
> > >> > > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they
> > are
> > >> > > >>>>>> already
> > >> > > >>>>>>>>>>>>>> co-partitioned.
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> -Jay
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> > >> > > >>>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> > >> > > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> > >> > > >> other
> > >> > > >>>>>>>> platforms
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> spark and pig.
> > >> > > >>>>>>>>>>>>>>> https://spark.apache.org/docs/
> 0.6.2/api/core/spark/
> > >> > > >>>>>>>>>>>>> PairRDDFunctions.html
> > >> > > >>>>>>>>>>>>>>>
> > https://www.tutorialspoint.com/apache_pig/apache_pig_
> > >> > > >>>>>>>>>>>>> cogroup_operator.htm
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> > >> > > >>>>> damian.guy@gmail.com>
> > >> > > >>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> > >> > > >> outer
> > >> > > >>>>>> join?
> > >> > > >>>>>>>> So
> > >> > > >>>>>>>>> an
> > >> > > >>>>>>>>>>>>> input
> > >> > > >>>>>>>>>>>>>>>> on any stream will always produce a new
> aggregated
> > >> > > >>> value
> > >> > > >>>>> -
> > >> > > >>>>>> is
> > >> > > >>>>>>>>> that
> > >> > > >>>>>>>>>>>>>>> correct?
> > >> > > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> > >> > > >> current
> > >> > > >>>>>> value,
> > >> > > >>>>>>>>>>>>> aggregates
> > >> > > >>>>>>>>>>>>>>>> and forwards the result.
> > >> > > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit
> > more,
> > >> > > >>>>> but it
> > >> > > >>>>>>>>> seems
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> it
> > >> > > >>>>>>>>>>>>>>>> could be a useful optimization.
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> > >> > > >>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> I sure can. I have added the following
> description
> > >> > > >> to
> > >> > > >>> my
> > >> > > >>>>>>>> KIP. If
> > >> > > >>>>>>>>>>> this
> > >> > > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some
> more
> > >> > > >>> time
> > >> > > >>>>> to
> > >> > > >>>>>>>>> build a
> > >> > > >>>>>>>>>>>>>>>> diagram
> > >> > > >>>>>>>>>>>>>>>>> and make more of a step by step description:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Example with Current API:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic1").
> groupByKey().aggregate(
> > >> > > >>>>>> initializer1
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator1,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic2").
> groupByKey().aggregate(
> > >> > > >>>>>> initializer2
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator2,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> > >> > > >>>>>>>>>>>>>>>>> builder.stream("topic3").
> groupByKey().aggregate(
> > >> > > >>>>>> initializer3
> > >> > > >>>>>>>> ,
> > >> > > >>>>>>>>>>>>>>>> aggregator3,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> table1.outerJoin(table2,
> > >> > > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> > >> > > >>>>> joinerOneTwoAndThree);
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores,
> > requires
> > >> > > >> 3
> > >> > > >>>>>>>>>> initializers,
> > >> > > >>>>>>>>>>>>>>> and 3
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to
> > user
> > >> > > >> to
> > >> > > >>>>>> define
> > >> > > >>>>>>>>> what
> > >> > > >>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2,
> V3).
> > >> > > >> They
> > >> > > >>>>> are
> > >> > > >>>>>>>> left
> > >> > > >>>>>>>>>>> with a
> > >> > > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all
> > the
> > >> > > >>>>> same
> > >> > > >>>>>> as
> > >> > > >>>>>>>> CG
> > >> > > >>>>>>>>>> and
> > >> > > >>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> two
> > >> > > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make
> them
> > >> > > >>>>>>>> intermediate
> > >> > > >>>>>>>>>>> states
> > >> > > >>>>>>>>>>>>>>>> such
> > >> > > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> > >> > > >> joiners
> > >> > > >>>>> use
> > >> > > >>>>>>>> those
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>> build
> > >> > > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something
> > the
> > >> > > >>> user
> > >> > > >>>>>>>> could
> > >> > > >>>>>>>>>> avoid
> > >> > > >>>>>>>>>>>>>>>>> thinking about with this KIP.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> > > >> will
> > >> > > >>>>> first
> > >> > > >>>>>>>> go
> > >> > > >>>>>>>>>>> through
> > >> > > >>>>>>>>>>>>>>> a
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate
> > from
> > >> > > >>>>>>>> storeName1.
> > >> > > >>>>>>>>>> It
> > >> > > >>>>>>>>>>>>>>> will
> > >> > > >>>>>>>>>>>>>>>>> produce this in the form of the first
> intermediate
> > >> > > >>> value
> > >> > > >>>>>> and
> > >> > > >>>>>>>> get
> > >> > > >>>>>>>>>>> sent
> > >> > > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will
> look
> > >> > > >> up
> > >> > > >>>>> the
> > >> > > >>>>>>>>> current
> > >> > > >>>>>>>>>>>>> value
> > >> > > >>>>>>>>>>>>>>>> of
> > >> > > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first
> > joiner
> > >> > > >> to
> > >> > > >>>>>>>> calculate
> > >> > > >>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> second
> > >> > > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> > >> > > >>> additional
> > >> > > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> > >> > > >>> current
> > >> > > >>>>>>>> value of
> > >> > > >>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>> key
> > >> > > >>>>>>>>>>>>>>>> in
> > >> > > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build
> the
> > >> > > >>> final
> > >> > > >>>>>>>>> aggregate
> > >> > > >>>>>>>>>>>>>>> value.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> If you think through all possibilities for
> > incoming
> > >> > > >>>>> topics
> > >> > > >>>>>>>> you
> > >> > > >>>>>>>>>> will
> > >> > > >>>>>>>>>>>>> see
> > >> > > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through
> all
> > >> > > >>> three
> > >> > > >>>>>>>> stores
> > >> > > >>>>>>>>>> are
> > >> > > >>>>>>>>>>>>>>>> queried
> > >> > > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this
> creates
> > N
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1)
> KTableKTableOuterJoins,
> > >> > > >> and
> > >> > > >>>>> N-1
> > >> > > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Example with Proposed API:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> > >> > > >>>>> builder.stream("topic1").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> > >> > > >>>>> builder.stream("topic2").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> > >> > > >>>>> builder.stream("topic3").
> > >> > > >>>>>>>>>>>>>>> groupByKey();
> > >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> > >> > > >>> grouped1.cogroup(initializer1,
> > >> > > >>>>>>>>>>> aggregator1,
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> > >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> > >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> > >> > > >>>>>>>>>>>>>>>>>      .aggregate();
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore,
> > requires 1
> > >> > > >>>>>>>>> initializer,
> > >> > > >>>>>>>>>>> and
> > >> > > >>>>>>>>>>>>> 1
> > >> > > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry
> > about
> > >> > > >>> the
> > >> > > >>>>>>>>>>> intermediate
> > >> > > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think
> > about
> > >> > > >>> is
> > >> > > >>>>> how
> > >> > > >>>>>>>> each
> > >> > > >>>>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> > > >> will
> > >> > > >>>>> first
> > >> > > >>>>>>>> go
> > >> > > >>>>>>>>>>> through
> > >> > > >>>>>>>>>>>>>>> a
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate
> > from
> > >> > > >>>>>>>> storeName1.
> > >> > > >>>>>>>>>> It
> > >> > > >>>>>>>>>>>>>>> will
> > >> > > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update
> > the
> > >> > > >>>>> store
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> pass
> > >> > > >>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>> new
> > >> > > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through
> the
> > >> > > >>>>>>>> KStreamCogroup
> > >> > > >>>>>>>>>>> which
> > >> > > >>>>>>>>>>>>>>> is
> > >> > > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and
> you
> > >> > > >> are
> > >> > > >>>>> done.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> > >> > > >> will
> > >> > > >>>>> only
> > >> > > >>>>>>>> every
> > >> > > >>>>>>>>>>>>>>> create N
> > >> > > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax
> <
> > >> > > >>>>>>>>>>>>> matthias@confluent.io
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> Kyle,
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little
> > slow,
> > >> > > >>>>> but I
> > >> > > >>>>>>>> could
> > >> > > >>>>>>>>>> not
> > >> > > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> > >> > > >>> concrete
> > >> > > >>>>>>>> example,
> > >> > > >>>>>>>>>>> like
> > >> > > >>>>>>>>>>>>>>> 3
> > >> > > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected
> > result),
> > >> > > >>> and
> > >> > > >>>>>> show
> > >> > > >>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>> difference between current way to to implement
> it
> > >> > > >> and
> > >> > > >>>>> the
> > >> > > >>>>>>>>>> proposed
> > >> > > >>>>>>>>>>>>>>> API?
> > >> > > >>>>>>>>>>>>>>>>>> This could also cover the internal processing
> to
> > >> > > >> see
> > >> > > >>>>> what
> > >> > > >>>>>>>> store
> > >> > > >>>>>>>>>>> calls
> > >> > > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you
> propose,
> > >> > > >> and
> > >> > > >>> it
> > >> > > >>>>>>>> would
> > >> > > >>>>>>>>>> help
> > >> > > >>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>> understand it better.
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> Thanks a lot!
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> -Matthias
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> > >> > > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found
> > here.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> > >> > > >> classes
> > >> > > >>>>> and
> > >> > > >>>>>> get
> > >> > > >>>>>>>>>> around
> > >> > > >>>>>>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> > >> > > >> additions.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> > >> > > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> > >> > > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> > >> > > >>>>>>>>> repartitionIfRequired
> > >> > > >>>>>>>>>>>>>>>> method.
> > >> > > >>>>>>>>>>>>>>>>> I
> > >> > > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more
> than
> > >> > > >> one
> > >> > > >>>>>>>> grouped
> > >> > > >>>>>>>>>>> stream
> > >> > > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> > >> > > >> name
> > >> > > >>>>> that
> > >> > > >>>>>>>>> someone
> > >> > > >>>>>>>>>>>>>>> can
> > >> > > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow
> > it
> > >> > > >> to
> > >> > > >>>>> fall
> > >> > > >>>>>>>> back
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped
> > tables?
> > >> > > >>> This
> > >> > > >>>>>>>> would
> > >> > > >>>>>>>>> be
> > >> > > >>>>>>>>>>>>>>>> pretty
> > >> > > >>>>>>>>>>>>>>>>>> easy
> > >> > > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing
> session
> > >> > > >>> stores
> > >> > > >>>>>> and
> > >> > > >>>>>>>>>>> windowed
> > >> > > >>>>>>>>>>>>>>>>>> stores
> > >> > > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate
> and
> > >> > > >>>>>>>>>>>>>>> KTableWindowAggregate
> > >> > > >>>>>>>>>>>>>>>>>>> implementations.
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>> Kyle
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> > >> > > >>>>>>>>> eno.thereska@gmail.com>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> Eno
> > >> > > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> > >> > > >>>>>>>>> damian.guy@gmail.com>
> > >> > > >>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i
> haven't
> > >> > > >> had
> > >> > > >>>>> the
> > >> > > >>>>>>>>> chance
> > >> > > >>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>> look
> > >> > > >>>>>>>>>>>>>>>>>> at
> > >> > > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to
> > look
> > >> > > >>>>> into
> > >> > > >>>>>> it
> > >> > > >>>>>>>>>>>>>>> tomorrow.
> > >> > > >>>>>>>>>>>>>>>>> For
> > >> > > >>>>>>>>>>>>>>>>>>>> the
> > >> > > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against
> > kafka
> > >> > > >>>>> trunk
> > >> > > >>>>>>>> and
> > >> > > >>>>>>>>>> mark
> > >> > > >>>>>>>>>>>>>>> it
> > >> > > >>>>>>>>>>>>>>>> as
> > >> > > >>>>>>>>>>>>>>>>>>>> WIP?
> > >> > > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have
> > done.
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>>>> Damian
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> > >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> > >> > > >> some
> > >> > > >>>>>>>> attention
> > >> > > >>>>>>>>> to
> > >> > > >>>>>>>>>>> my
> > >> > > >>>>>>>>>>>>>>>> KIP
> > >> > > >>>>>>>>>>>>>>>>>> as
> > >> > > >>>>>>>>>>>>>>>>>>>> I
> > >> > > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> > >> > > >> This
> > >> > > >>>>> is my
> > >> > > >>>>>>>>> first
> > >> > > >>>>>>>>>>> KIP
> > >> > > >>>>>>>>>>>>>>>> and
> > >> > > >>>>>>>>>>>>>>>>>> my
> > >> > > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so
> > I'm
> > >> > > >>>>> sure I
> > >> > > >>>>>>>> did
> > >> > > >>>>>>>>>>>>>>>> something
> > >> > > >>>>>>>>>>>>>>>>>>>> wrong.
> > >> > > >>>>>>>>>>>>>>>>>>>>>> ;)
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> > >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> > >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> > >> > > >> discussion
> > >> > > >>>>> about
> > >> > > >>>>>>>>> adding
> > >> > > >>>>>>>>>>>>>>>>> cogroup
> > >> > > >>>>>>>>>>>>>>>>>> to
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > >> > > >>>>>> confluence/display/KAFKA/KIP-
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation
> here:
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >> > > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> > >> > > >>>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>>>
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>>
> > >> > > >>>>>>>>>>
> > >> > > >>>>>>>>>
> > >> > > >>>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>>
> > >> > > >>>>>>> --
> > >> > > >>>>>>> -- Guozhang
> > >> > > >>>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>>
> > >> > > >>>>>> --
> > >> > > >>>>>> -- Guozhang
> > >> > > >>>>>>
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>>
> > >> > > >>>>> --
> > >> > > >>>>> -- Guozhang
> > >> > > >>>>>
> > >> > > >>>>
> > >> > > >>>
> > >> > > >>
> > >> > >
> > >> > >
> > >> >
> > >> >
> > >> > --
> > >> > -- Guozhang
> > >> >
> > >>
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Xavier Léauté <xa...@confluent.io>.
Another reason for the serde not to be in the first cogroup call, is that
the serde should not be required if you pass a StateStoreSupplier to
aggregate()

Regarding the aggregated type <T> I don't the why initializer should be
favored over aggregator to define the type. In my mind separating the
initializer into the last aggregate call clearly indicates that the
initializer is independent of any of the aggregators or streams and that we
don't wait for grouped1 events to initialize the co-group.

On Thu, Jun 8, 2017 at 11:14 AM Guozhang Wang <wa...@gmail.com> wrote:

> On a second thought... This is the current proposal API
>
>
> ```
>
> <T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer, final
> Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
> aggValueSerde)
>
> ```
>
>
> If we do not have the initializer in the first co-group it might be a bit
> awkward for users to specify the aggregator that returns a typed <T> value?
> Maybe it is still better to put these two functions in the same api?
>
>
>
> Guozhang
>
> On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > This suggestion lgtm. I would vote for the first alternative than adding
> > it to the `KStreamBuilder` though.
> >
> > On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
> > wrote:
> >
> >> I have a minor suggestion to make the API a little bit more symmetric.
> >> I feel it would make more sense to move the initializer and serde to the
> >> final aggregate statement, since the serde only applies to the state
> >> store,
> >> and the initializer doesn't bear any relation to the first group in
> >> particular. It would end up looking like this:
> >>
> >> KTable<K, CG> cogrouped =
> >>     grouped1.cogroup(aggregator1)
> >>             .cogroup(grouped2, aggregator2)
> >>             .cogroup(grouped3, aggregator3)
> >>             .aggregate(initializer1, aggValueSerde, storeName1);
> >>
> >> Alternatively, we could move the the first cogroup() method to
> >> KStreamBuilder, similar to how we have .merge()
> >> and end up with an api that would be even more symmetric.
> >>
> >> KStreamBuilder.cogroup(grouped1, aggregator1)
> >>               .cogroup(grouped2, aggregator2)
> >>               .cogroup(grouped3, aggregator3)
> >>               .aggregate(initializer1, aggValueSerde, storeName1);
> >>
> >> This doesn't have to be a blocker, but I thought it would make the API
> >> just
> >> a tad cleaner.
> >>
> >> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >> > Kyle,
> >> >
> >> > Thanks a lot for the updated KIP. It looks good to me.
> >> >
> >> >
> >> > Guozhang
> >> >
> >> >
> >> > On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com>
> wrote:
> >> >
> >> > > This makes much more sense to me. +1
> >> > >
> >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> >> winkelman.kyle@gmail.com>
> >> > > wrote:
> >> > > >
> >> > > > I have updated the KIP and my PR. Let me know what you think.
> >> > > > To created a cogrouped stream just call cogroup on a
> KgroupedStream
> >> and
> >> > > > supply the initializer, aggValueSerde, and an aggregator. Then
> >> continue
> >> > > > adding kgroupedstreams and aggregators. Then call one of the many
> >> > > aggregate
> >> > > > calls to create a KTable.
> >> > > >
> >> > > > Thanks,
> >> > > > Kyle
> >> > > >
> >> > > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com>
> wrote:
> >> > > >
> >> > > >> Hi Kyle,
> >> > > >>
> >> > > >> Thanks for the update. I think just one initializer makes sense
> as
> >> it
> >> > > >> should only be called once per key and generally it is just going
> >> to
> >> > > create
> >> > > >> a new instance of whatever the Aggregate class is.
> >> > > >>
> >> > > >> Cheers,
> >> > > >> Damian
> >> > > >>
> >> > > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> >> winkelman.kyle@gmail.com
> >> > >
> >> > > >> wrote:
> >> > > >>
> >> > > >>> Hello all,
> >> > > >>>
> >> > > >>> I have spent some more time on this and the best alternative I
> >> have
> >> > > come
> >> > > >> up
> >> > > >>> with is:
> >> > > >>> KGroupedStream has a single cogroup call that takes an
> initializer
> >> > and
> >> > > an
> >> > > >>> aggregator.
> >> > > >>> CogroupedKStream has a cogroup call that takes additional
> >> > groupedStream
> >> > > >>> aggregator pairs.
> >> > > >>> CogroupedKStream has multiple aggregate methods that create the
> >> > > different
> >> > > >>> stores.
> >> > > >>>
> >> > > >>> I plan on updating the kip but I want people's input on if we
> >> should
> >> > > have
> >> > > >>> the initializer be passed in once at the beginning or if we
> should
> >> > > >> instead
> >> > > >>> have the initializer be required for each call to one of the
> >> > aggregate
> >> > > >>> calls. The first makes more sense to me but doesnt allow the
> user
> >> to
> >> > > >>> specify different initializers for different tables.
> >> > > >>>
> >> > > >>> Thanks,
> >> > > >>> Kyle
> >> > > >>>
> >> > > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> >> winkelman.kyle@gmail.com>
> >> > > >>> wrote:
> >> > > >>>
> >> > > >>>> Yea I really like that idea I'll see what I can do to update
> the
> >> kip
> >> > > >> and
> >> > > >>>> my pr when I have some time. I'm not sure how well creating the
> >> > > >>>> kstreamaggregates will go though because at that point I will
> >> have
> >> > > >> thrown
> >> > > >>>> away the type of the values. It will be type safe I just may
> >> need to
> >> > > >> do a
> >> > > >>>> little forcing.
> >> > > >>>>
> >> > > >>>> Thanks,
> >> > > >>>> Kyle
> >> > > >>>>
> >> > > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com>
> >> > wrote:
> >> > > >>>>
> >> > > >>>>> Kyle,
> >> > > >>>>>
> >> > > >>>>> Thanks for the explanations, my previous read on the wiki
> >> examples
> >> > > was
> >> > > >>>>> wrong.
> >> > > >>>>>
> >> > > >>>>> So I guess my motivation should be "reduced" to: can we move
> the
> >> > > >> window
> >> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> >> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> >> > > >>>>>
> >> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream
> from
> >> 3
> >> > to
> >> > > >> 2.
> >> > > >>>>> 2. major: this is for extensibility of the APIs, and since we
> >> are
> >> > > >>> removing
> >> > > >>>>> the "Evolving" annotations on Streams it may be harder to
> >> change it
> >> > > >>> again
> >> > > >>>>> in the future. The extended use cases are that people wanted
> to
> >> > have
> >> > > >>>>> windowed running aggregates on different granularities, e.g.
> >> "give
> >> > me
> >> > > >>> the
> >> > > >>>>> counts per-minute, per-hour, per-day and per-week", and today
> in
> >> > DSL
> >> > > >> we
> >> > > >>>>> need to specify that case in multiple aggregate operators,
> which
> >> > gets
> >> > > >> a
> >> > > >>>>> state store / changelog, etc. And it is possible to optimize
> it
> >> as
> >> > > >> well
> >> > > >>> to
> >> > > >>>>> a single state store. Its implementation would be tricky as
> you
> >> > need
> >> > > >> to
> >> > > >>>>> contain different lengthed windows within your window store
> but
> >> > just
> >> > > >>> from
> >> > > >>>>> the public API point of view, it could be specified as:
> >> > > >>>>>
> >> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >> > > >>>>> "state-store-name");
> >> > > >>>>>
> >> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> >> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> >> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
> >> > > >>>>>
> >> > > >>>>> while underlying we are only using a single store
> >> > "state-store-name"
> >> > > >> for
> >> > > >>>>> it.
> >> > > >>>>>
> >> > > >>>>>
> >> > > >>>>> Although this feature is out of the scope of this KIP, I'd
> like
> >> to
> >> > > >>> discuss
> >> > > >>>>> if we can "leave the door open" to make such changes without
> >> > > modifying
> >> > > >>> the
> >> > > >>>>> public APIs .
> >> > > >>>>>
> >> > > >>>>> Guozhang
> >> > > >>>>>
> >> > > >>>>>
> >> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> >> > > >>> winkelman.kyle@gmail.com
> >> > > >>>>>>
> >> > > >>>>> wrote:
> >> > > >>>>>
> >> > > >>>>>> I allow defining a single window/sessionwindow one time when
> >> you
> >> > > >> make
> >> > > >>>>> the
> >> > > >>>>>> cogroup call from a KGroupedStream. From then on you are
> using
> >> the
> >> > > >>>>> cogroup
> >> > > >>>>>> call from with in CogroupedKStream which doesnt accept any
> >> > > >> additional
> >> > > >>>>>> windows/sessionwindows.
> >> > > >>>>>>
> >> > > >>>>>> Is this what you meant by your question or did I
> misunderstand?
> >> > > >>>>>>
> >> > > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wangguoz@gmail.com
> >
> >> > > >> wrote:
> >> > > >>>>>>
> >> > > >>>>>> Another question that came to me is on "window alignment":
> from
> >> > the
> >> > > >>> KIP
> >> > > >>>>> it
> >> > > >>>>>> seems you are allowing users to specify a (potentially
> >> different)
> >> > > >>> window
> >> > > >>>>>> spec in each co-grouped input stream. So if these window
> specs
> >> are
> >> > > >>>>>> different how should we "align" them with different input
> >> > streams? I
> >> > > >>>>> think
> >> > > >>>>>> it is more natural to only specify on window spec in the
> >> > > >>>>>>
> >> > > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> And remove it from the cogroup() functions. WDYT?
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> Guozhang
> >> > > >>>>>>
> >> > > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> >> > wangguoz@gmail.com>
> >> > > >>>>> wrote:
> >> > > >>>>>>
> >> > > >>>>>>> Thanks for the proposal Kyle, this is a quite common use
> case
> >> to
> >> > > >>>>> support
> >> > > >>>>>>> such multi-way table join (i.e. N source tables with N
> >> aggregate
> >> > > >>> func)
> >> > > >>>>>> with
> >> > > >>>>>>> a single store and N+1 serdes, I have seen lots of people
> >> using
> >> > > >> the
> >> > > >>>>>>> low-level PAPI to achieve this goal.
> >> > > >>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >> > > >>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>> wrote:
> >> > > >>>>>>>
> >> > > >>>>>>>> I like your point about not handling other cases such as
> >> count
> >> > > >> and
> >> > > >>>>>> reduce.
> >> > > >>>>>>>>
> >> > > >>>>>>>> I think that reduce may not make sense because reduce
> assumes
> >> > > >> that
> >> > > >>>>> the
> >> > > >>>>>>>> input values are the same as the output values. With
> cogroup
> >> > > >> there
> >> > > >>>>> may
> >> > > >>>>>> be
> >> > > >>>>>>>> multiple different input types and then your output type
> >> cant be
> >> > > >>>>>> multiple
> >> > > >>>>>>>> different things. In the case where you have all matching
> >> value
> >> > > >>> types
> >> > > >>>>>> you
> >> > > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> >> > > >>>>>>>>
> >> > > >>>>>>>> As for count I think it is possible to call count on all
> the
> >> > > >>>>> individual
> >> > > >>>>>>>> grouped streams and then do joins. Otherwise we could maybe
> >> make
> >> > > >> a
> >> > > >>>>>> special
> >> > > >>>>>>>> call in groupedstream for this case. Because in this case
> we
> >> > dont
> >> > > >>>>> need
> >> > > >>>>>> to
> >> > > >>>>>>>> do type checking on the values. It could be similar to the
> >> > > >> current
> >> > > >>>>> count
> >> > > >>>>>>>> methods but accept a var args of additonal grouped streams
> as
> >> > > >> well
> >> > > >>>>> and
> >> > > >>>>>>>> make
> >> > > >>>>>>>> sure they have a key type of K.
> >> > > >>>>>>>>
> >> > > >>>>>>>> The way I have put the kip together is to ensure that we do
> >> type
> >> > > >>>>>> checking.
> >> > > >>>>>>>> I don't see a way we could group them all first and then
> >> make a
> >> > > >>> call
> >> > > >>>>> to
> >> > > >>>>>>>> count, reduce, or aggregate because with aggregate they
> would
> >> > > >> need
> >> > > >>> to
> >> > > >>>>>> pass
> >> > > >>>>>>>> a list of aggregators and we would have no way of type
> >> checking
> >> > > >>> that
> >> > > >>>>>> they
> >> > > >>>>>>>> match the grouped streams.
> >> > > >>>>>>>>
> >> > > >>>>>>>> Thanks,
> >> > > >>>>>>>> Kyle
> >> > > >>>>>>>>
> >> > > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
> >> xavier@confluent.io>
> >> > > >>>>> wrote:
> >> > > >>>>>>>>
> >> > > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a
> very
> >> > > >>> useful
> >> > > >>>>>>>>> addition and wanted to provide an additional use-case and
> >> some
> >> > > >>> more
> >> > > >>>>>>>>> comments.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> This is actually a very common analytics use-case in the
> >> > > >> ad-tech
> >> > > >>>>>>>> industry.
> >> > > >>>>>>>>> The typical setup will have an auction stream, an
> impression
> >> > > >>>>> stream,
> >> > > >>>>>>>> and a
> >> > > >>>>>>>>> click stream. Those three streams need to be combined to
> >> > > >> compute
> >> > > >>>>>>>> aggregate
> >> > > >>>>>>>>> statistics (e.g. impression statistics, and click-through
> >> > > >> rates),
> >> > > >>>>>> since
> >> > > >>>>>>>>> most of the attributes of interest are only present the
> >> auction
> >> > > >>>>>> stream.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> A simple way to do this is to co-group all the streams by
> >> the
> >> > > >>>>> auction
> >> > > >>>>>>>> key,
> >> > > >>>>>>>>> and process updates to the co-group as events for each
> >> stream
> >> > > >>> come
> >> > > >>>>> in,
> >> > > >>>>>>>>> keeping only one value from each stream before sending
> >> > > >> downstream
> >> > > >>>>> for
> >> > > >>>>>>>>> further processing / aggregation.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> One could view the result of that co-group operation as a
> >> > > >>> "KTable"
> >> > > >>>>>> with
> >> > > >>>>>>>>> multiple values per key. The key being the grouping key,
> and
> >> > > >> the
> >> > > >>>>>> values
> >> > > >>>>>>>>> consisting of one value per stream.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> What I like about Kyle's approach is that allows elegant
> >> > > >>>>> co-grouping
> >> > > >>>>>> of
> >> > > >>>>>>>>> multiple streams without having to worry about the number
> of
> >> > > >>>>> streams,
> >> > > >>>>>>>> and
> >> > > >>>>>>>>> avoids dealing with Tuple types or other generic
> interfaces
> >> > > >> that
> >> > > >>>>> could
> >> > > >>>>>>>> get
> >> > > >>>>>>>>> messy if we wanted to preserve all the value types in the
> >> > > >>> resulting
> >> > > >>>>>>>>> co-grouped stream.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> My only concern is that we only allow the cogroup +
> >> aggregate
> >> > > >>>>> combined
> >> > > >>>>>>>>> operation. This forces the user to build their own tuple
> >> > > >>>>> serialization
> >> > > >>>>>>>>> format if they want to preserve the individual input
> stream
> >> > > >>> values
> >> > > >>>>> as
> >> > > >>>>>> a
> >> > > >>>>>>>>> group. It also deviates quite a bit from our approach in
> >> > > >>>>>> KGroupedStream
> >> > > >>>>>>>>> which offers other operations, such as count and reduce,
> >> which
> >> > > >>>>> should
> >> > > >>>>>>>> also
> >> > > >>>>>>>>> be applicable to a co-grouped stream.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> Overall I still think this is a really useful addition,
> but
> >> I
> >> > > >>> feel
> >> > > >>>>> we
> >> > > >>>>>>>>> haven't spend much time trying to explore alternative DSLs
> >> that
> >> > > >>>>> could
> >> > > >>>>>>>> maybe
> >> > > >>>>>>>>> generalize better or match our existing syntax more
> closely.
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> >> > > >>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>>>
> >> > > >>>>>>>>> wrote:
> >> > > >>>>>>>>>
> >> > > >>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
> >> > > >>> streams
> >> > > >>>>>> realm
> >> > > >>>>>>>>> that
> >> > > >>>>>>>>>> I should reach out to for input?
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>> I believe Damian Guy is still planning on reviewing this
> >> more
> >> > > >>> in
> >> > > >>>>>> depth
> >> > > >>>>>>>>> so I
> >> > > >>>>>>>>>> will wait for his inputs before continuing.
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> >> > > >> eno.thereska@gmail.com
> >> > > >>>>
> >> > > >>>>>>>> wrote:
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>>> Thanks Kyle, good arguments.
> >> > > >>>>>>>>>>>
> >> > > >>>>>>>>>>> Eno
> >> > > >>>>>>>>>>>
> >> > > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> >> > > >>>>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> *- minor: could you add an exact example (similar to
> what
> >> > > >>>>> Jay’s
> >> > > >>>>>>>>> example
> >> > > >>>>>>>>>>> is,
> >> > > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> >> > > >>>>>> concrete?*
> >> > > >>>>>>>>>>>> I have added a more concrete example to the KIP.
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> *- my main concern is that we’re exposing this
> >> > > >> optimization
> >> > > >>>>> to
> >> > > >>>>>> the
> >> > > >>>>>>>>> DSL.
> >> > > >>>>>>>>>>> In
> >> > > >>>>>>>>>>>> an ideal world, an optimizer would take the existing
> DSL
> >> > > >>> and
> >> > > >>>>> do
> >> > > >>>>>>>> the
> >> > > >>>>>>>>>> right
> >> > > >>>>>>>>>>>> thing under the covers (create just one state store,
> >> > > >>> arrange
> >> > > >>>>> the
> >> > > >>>>>>>>> nodes
> >> > > >>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> >> > > >>>>> pieces
> >> > > >>>>>>>>> (group,
> >> > > >>>>>>>>>>>> aggregate, join) that this proposal groups together.
> I’d
> >> > > >>>>> like to
> >> > > >>>>>>>> hear
> >> > > >>>>>>>>>>> your
> >> > > >>>>>>>>>>>> thoughts on whether it’s possible to do this
> optimization
> >> > > >>>>> with
> >> > > >>>>>> the
> >> > > >>>>>>>>>>> current
> >> > > >>>>>>>>>>>> DSL, at the topology builder level.*
> >> > > >>>>>>>>>>>> You would have to make a lot of checks to understand if
> >> > > >> it
> >> > > >>> is
> >> > > >>>>>> even
> >> > > >>>>>>>>>>> possible
> >> > > >>>>>>>>>>>> to make this optimization:
> >> > > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> >> > > >>>>>>>>>>>> 2. None of the intermediate KTables are used for
> anything
> >> > > >>>>> else.
> >> > > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This may
> be
> >> > > >>>>>>>> impossible
> >> > > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> >> > > >>> topology
> >> > > >>>>> has
> >> > > >>>>>>>>>> already
> >> > > >>>>>>>>>>>> been built.)
> >> > > >>>>>>>>>>>> You would then need to make decisions during the
> >> > > >>>>> optimization:
> >> > > >>>>>>>>>>>> 1. Your new initializer would the composite of all the
> >> > > >>>>>> individual
> >> > > >>>>>>>>>>>> initializers and the valueJoiners.
> >> > > >>>>>>>>>>>> 2. I am having a hard time thinking about how you would
> >> > > >>> turn
> >> > > >>>>> the
> >> > > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> >> > > >> would
> >> > > >>>>> work
> >> > > >>>>>> on
> >> > > >>>>>>>>> the
> >> > > >>>>>>>>>>>> final object, but this may be possible.
> >> > > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> >> > > >> would
> >> > > >>>>> be
> >> > > >>>>>> for
> >> > > >>>>>>>>> the
> >> > > >>>>>>>>>>>> aggregate values. None of the declared ones would be
> >> > > >>>>> guaranteed
> >> > > >>>>>> to
> >> > > >>>>>>>>> hold
> >> > > >>>>>>>>>>> the
> >> > > >>>>>>>>>>>> final object. This would mean you must created a new
> >> > > >> state
> >> > > >>>>> store
> >> > > >>>>>>>> and
> >> > > >>>>>>>>>> not
> >> > > >>>>>>>>>>>> created any of the declared ones.
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> The main argument I have against it is even if it could
> >> > > >> be
> >> > > >>>>> done
> >> > > >>>>>> I
> >> > > >>>>>>>>> don't
> >> > > >>>>>>>>>>>> know that we would want to have this be an optimization
> >> > > >> in
> >> > > >>>>> the
> >> > > >>>>>>>>>> background
> >> > > >>>>>>>>>>>> because the user would still be required to think about
> >> > > >> all
> >> > > >>>>> of
> >> > > >>>>>> the
> >> > > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> >> > > >> about
> >> > > >>>>> if
> >> > > >>>>>>>> they
> >> > > >>>>>>>>>> only
> >> > > >>>>>>>>>>>> care about the final object.
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> >> > > >>> should
> >> > > >>>>> be
> >> > > >>>>>>>> part
> >> > > >>>>>>>>> of
> >> > > >>>>>>>>>>> the
> >> > > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> >> > > >> want
> >> > > >>> to
> >> > > >>>>>>>> allow
> >> > > >>>>>>>>>>> people
> >> > > >>>>>>>>>>>> to join more than 2 or more streams in an easy way.
> Right
> >> > > >>>>> now I
> >> > > >>>>>>>> don't
> >> > > >>>>>>>>>>> think
> >> > > >>>>>>>>>>>> we give them ways of handling this use case easily.
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> *-I think there will be scope for several such
> >> > > >>> optimizations
> >> > > >>>>> in
> >> > > >>>>>>>> the
> >> > > >>>>>>>>>>> future
> >> > > >>>>>>>>>>>> and perhaps at some point we need to think about
> >> > > >> decoupling
> >> > > >>>>> the
> >> > > >>>>>>>> 1:1
> >> > > >>>>>>>>>>> mapping
> >> > > >>>>>>>>>>>> from the DSL into the physical topology.*
> >> > > >>>>>>>>>>>> I would argue that cogroup is not just an optimization
> it
> >> > > >>> is
> >> > > >>>>> a
> >> > > >>>>>> new
> >> > > >>>>>>>>> way
> >> > > >>>>>>>>>>> for
> >> > > >>>>>>>>>>>> the users to look at accomplishing a problem that
> >> > > >> requires
> >> > > >>>>>>>> multiple
> >> > > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> >> > > >> think
> >> > > >>>>>> users
> >> > > >>>>>>>>>> should
> >> > > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
> >> > > >>> their
> >> > > >>>>>>>>>>> initializers,
> >> > > >>>>>>>>>>>> serdes and stores if all they care about is the final
> >> > > >>> object.
> >> > > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> >> > > >> supply
> >> > > >>>>>>>> additional
> >> > > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> >> > > >> single
> >> > > >>>>>>>> grouped
> >> > > >>>>>>>>>>> stream
> >> > > >>>>>>>>>>>> making an aggregate call. This case is what I view an
> >> > > >>>>>> optimization
> >> > > >>>>>>>>> as,
> >> > > >>>>>>>>>> we
> >> > > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
> >> > > >>> just
> >> > > >>>>> a
> >> > > >>>>>>>> call
> >> > > >>>>>>>>> to
> >> > > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> >> > > >>>>>>>> KGroupedStream#cogroup.
> >> > > >>>>>>>>> (I
> >> > > >>>>>>>>>>>> would prefer to just write a warning saying that this
> is
> >> > > >>> not
> >> > > >>>>> how
> >> > > >>>>>>>>>> cogroup
> >> > > >>>>>>>>>>> is
> >> > > >>>>>>>>>>>> to be used.)
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> Thanks,
> >> > > >>>>>>>>>>>> Kyle
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> >> > > >>>>>>>> eno.thereska@gmail.com
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> Hi Kyle,
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> - minor: could you add an exact example (similar to
> what
> >> > > >>>>> Jay’s
> >> > > >>>>>>>>> example
> >> > > >>>>>>>>>>> is,
> >> > > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this
> super
> >> > > >>>>>> concrete?
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> - my main concern is that we’re exposing this
> >> > > >> optimization
> >> > > >>>>> to
> >> > > >>>>>> the
> >> > > >>>>>>>>> DSL.
> >> > > >>>>>>>>>>> In
> >> > > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing
> DSL
> >> > > >>>>> and do
> >> > > >>>>>>>> the
> >> > > >>>>>>>>>>> right
> >> > > >>>>>>>>>>>>> thing under the covers (create just one state store,
> >> > > >>> arrange
> >> > > >>>>>> the
> >> > > >>>>>>>>> nodes
> >> > > >>>>>>>>>>>>> etc). The original DSL had a bunch of small,
> composable
> >> > > >>>>> pieces
> >> > > >>>>>>>>> (group,
> >> > > >>>>>>>>>>>>> aggregate, join) that this proposal groups together.
> I’d
> >> > > >>>>> like
> >> > > >>>>>> to
> >> > > >>>>>>>>> hear
> >> > > >>>>>>>>>>> your
> >> > > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> >> > > >> optimization
> >> > > >>>>> with
> >> > > >>>>>>>> the
> >> > > >>>>>>>>>>> current
> >> > > >>>>>>>>>>>>> DSL, at the topology builder level.
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> I think there will be scope for several such
> >> > > >> optimizations
> >> > > >>>>> in
> >> > > >>>>>> the
> >> > > >>>>>>>>>> future
> >> > > >>>>>>>>>>>>> and perhaps at some point we need to think about
> >> > > >>> decoupling
> >> > > >>>>> the
> >> > > >>>>>>>> 1:1
> >> > > >>>>>>>>>>> mapping
> >> > > >>>>>>>>>>>>> from the DSL into the physical topology.
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>> Thanks
> >> > > >>>>>>>>>>>>> Eno
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> >> > > >> jay@confluent.io>
> >> > > >>>>>> wrote:
> >> > > >>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> >> > > >>> pretty
> >> > > >>>>>>>> common.
> >> > > >>>>>>>>> An
> >> > > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> >> > > >> customer
> >> > > >>>>>>>> profile"
> >> > > >>>>>>>>> use
> >> > > >>>>>>>>>>>>> case
> >> > > >>>>>>>>>>>>>> we often use. In that use case you have a dozen
> systems
> >> > > >>>>> each
> >> > > >>>>>> of
> >> > > >>>>>>>>> which
> >> > > >>>>>>>>>>> has
> >> > > >>>>>>>>>>>>>> some information about your customer (account
> details,
> >> > > >>>>>> settings,
> >> > > >>>>>>>>>>> billing
> >> > > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> >> > > >> etc).
> >> > > >>>>> Your
> >> > > >>>>>>>> goal
> >> > > >>>>>>>>> is
> >> > > >>>>>>>>>>> to
> >> > > >>>>>>>>>>>>>> join/munge these into a single profile record for
> each
> >> > > >>>>>> customer
> >> > > >>>>>>>>> that
> >> > > >>>>>>>>>>> has
> >> > > >>>>>>>>>>>>>> all the relevant info in a usable form and is
> >> > > >> up-to-date
> >> > > >>>>> with
> >> > > >>>>>>>> all
> >> > > >>>>>>>>> the
> >> > > >>>>>>>>>>>>>> source systems. If you implement that with kstreams
> as
> >> > > >> a
> >> > > >>>>>>>> sequence
> >> > > >>>>>>>>> of
> >> > > >>>>>>>>>>>>> joins
> >> > > >>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
> >> > > >>>>> tables.
> >> > > >>>>>>>> But
> >> > > >>>>>>>>>>> clearly
> >> > > >>>>>>>>>>>>>> you only need a single stage to group all these
> things
> >> > > >>> that
> >> > > >>>>>> are
> >> > > >>>>>>>>>> already
> >> > > >>>>>>>>>>>>>> co-partitioned. A distributed database would do this
> >> > > >>> under
> >> > > >>>>> the
> >> > > >>>>>>>>> covers
> >> > > >>>>>>>>>>>>> which
> >> > > >>>>>>>>>>>>>> is arguably better (at least when it does the right
> >> > > >>> thing)
> >> > > >>>>> and
> >> > > >>>>>>>>>> perhaps
> >> > > >>>>>>>>>>> we
> >> > > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
> >> > > >>>>>>>> partitioning
> >> > > >>>>>>>>> so
> >> > > >>>>>>>>>> we
> >> > > >>>>>>>>>>>>> may
> >> > > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they
> are
> >> > > >>>>>> already
> >> > > >>>>>>>>>>>>>> co-partitioned.
> >> > > >>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>> -Jay
> >> > > >>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> >> > > >>>>>>>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> >> > > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> >> > > >> other
> >> > > >>>>>>>> platforms
> >> > > >>>>>>>>>>> like
> >> > > >>>>>>>>>>>>>>> spark and pig.
> >> > > >>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> >> > > >>>>>>>>>>>>> PairRDDFunctions.html
> >> > > >>>>>>>>>>>>>>>
> https://www.tutorialspoint.com/apache_pig/apache_pig_
> >> > > >>>>>>>>>>>>> cogroup_operator.htm
> >> > > >>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> >> > > >>>>> damian.guy@gmail.com>
> >> > > >>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>> Hi Kyle,
> >> > > >>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> >> > > >> outer
> >> > > >>>>>> join?
> >> > > >>>>>>>> So
> >> > > >>>>>>>>> an
> >> > > >>>>>>>>>>>>> input
> >> > > >>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
> >> > > >>> value
> >> > > >>>>> -
> >> > > >>>>>> is
> >> > > >>>>>>>>> that
> >> > > >>>>>>>>>>>>>>> correct?
> >> > > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> >> > > >> current
> >> > > >>>>>> value,
> >> > > >>>>>>>>>>>>> aggregates
> >> > > >>>>>>>>>>>>>>>> and forwards the result.
> >> > > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit
> more,
> >> > > >>>>> but it
> >> > > >>>>>>>>> seems
> >> > > >>>>>>>>>>> like
> >> > > >>>>>>>>>>>>>>> it
> >> > > >>>>>>>>>>>>>>>> could be a useful optimization.
> >> > > >>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> >> > > >>>>>>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> I sure can. I have added the following description
> >> > > >> to
> >> > > >>> my
> >> > > >>>>>>>> KIP. If
> >> > > >>>>>>>>>>> this
> >> > > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
> >> > > >>> time
> >> > > >>>>> to
> >> > > >>>>>>>>> build a
> >> > > >>>>>>>>>>>>>>>> diagram
> >> > > >>>>>>>>>>>>>>>>> and make more of a step by step description:
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> Example with Current API:
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> >> > > >>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
> >> > > >>>>>> initializer1
> >> > > >>>>>>>> ,
> >> > > >>>>>>>>>>>>>>>> aggregator1,
> >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> >> > > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> >> > > >>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
> >> > > >>>>>> initializer2
> >> > > >>>>>>>> ,
> >> > > >>>>>>>>>>>>>>>> aggregator2,
> >> > > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> >> > > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> >> > > >>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
> >> > > >>>>>> initializer3
> >> > > >>>>>>>> ,
> >> > > >>>>>>>>>>>>>>>> aggregator3,
> >> > > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> >> > > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> >> > > >>>>> joinerOneTwoAndThree);
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores,
> requires
> >> > > >> 3
> >> > > >>>>>>>>>> initializers,
> >> > > >>>>>>>>>>>>>>> and 3
> >> > > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to
> user
> >> > > >> to
> >> > > >>>>>> define
> >> > > >>>>>>>>> what
> >> > > >>>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
> >> > > >> They
> >> > > >>>>> are
> >> > > >>>>>>>> left
> >> > > >>>>>>>>>>> with a
> >> > > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all
> the
> >> > > >>>>> same
> >> > > >>>>>> as
> >> > > >>>>>>>> CG
> >> > > >>>>>>>>>> and
> >> > > >>>>>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>> two
> >> > > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
> >> > > >>>>>>>> intermediate
> >> > > >>>>>>>>>>> states
> >> > > >>>>>>>>>>>>>>>> such
> >> > > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> >> > > >> joiners
> >> > > >>>>> use
> >> > > >>>>>>>> those
> >> > > >>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>> build
> >> > > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something
> the
> >> > > >>> user
> >> > > >>>>>>>> could
> >> > > >>>>>>>>>> avoid
> >> > > >>>>>>>>>>>>>>>>> thinking about with this KIP.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> >> > > >> will
> >> > > >>>>> first
> >> > > >>>>>>>> go
> >> > > >>>>>>>>>>> through
> >> > > >>>>>>>>>>>>>>> a
> >> > > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate
> from
> >> > > >>>>>>>> storeName1.
> >> > > >>>>>>>>>> It
> >> > > >>>>>>>>>>>>>>> will
> >> > > >>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
> >> > > >>> value
> >> > > >>>>>> and
> >> > > >>>>>>>> get
> >> > > >>>>>>>>>>> sent
> >> > > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
> >> > > >> up
> >> > > >>>>> the
> >> > > >>>>>>>>> current
> >> > > >>>>>>>>>>>>> value
> >> > > >>>>>>>>>>>>>>>> of
> >> > > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first
> joiner
> >> > > >> to
> >> > > >>>>>>>> calculate
> >> > > >>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>> second
> >> > > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> >> > > >>> additional
> >> > > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> >> > > >>> current
> >> > > >>>>>>>> value of
> >> > > >>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>> key
> >> > > >>>>>>>>>>>>>>>> in
> >> > > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
> >> > > >>> final
> >> > > >>>>>>>>> aggregate
> >> > > >>>>>>>>>>>>>>> value.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> If you think through all possibilities for
> incoming
> >> > > >>>>> topics
> >> > > >>>>>>>> you
> >> > > >>>>>>>>>> will
> >> > > >>>>>>>>>>>>> see
> >> > > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
> >> > > >>> three
> >> > > >>>>>>>> stores
> >> > > >>>>>>>>>> are
> >> > > >>>>>>>>>>>>>>>> queried
> >> > > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates
> N
> >> > > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
> >> > > >> and
> >> > > >>>>> N-1
> >> > > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> Example with Proposed API:
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> >> > > >>>>> builder.stream("topic1").
> >> > > >>>>>>>>>>>>>>> groupByKey();
> >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> >> > > >>>>> builder.stream("topic2").
> >> > > >>>>>>>>>>>>>>> groupByKey();
> >> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> >> > > >>>>> builder.stream("topic3").
> >> > > >>>>>>>>>>>>>>> groupByKey();
> >> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> >> > > >>> grouped1.cogroup(initializer1,
> >> > > >>>>>>>>>>> aggregator1,
> >> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> >> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> >> > > >>>>>>>>>>>>>>>>>      .aggregate();
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore,
> requires 1
> >> > > >>>>>>>>> initializer,
> >> > > >>>>>>>>>>> and
> >> > > >>>>>>>>>>>>> 1
> >> > > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry
> about
> >> > > >>> the
> >> > > >>>>>>>>>>> intermediate
> >> > > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think
> about
> >> > > >>> is
> >> > > >>>>> how
> >> > > >>>>>>>> each
> >> > > >>>>>>>>>>>>> stream
> >> > > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> >> > > >> will
> >> > > >>>>> first
> >> > > >>>>>>>> go
> >> > > >>>>>>>>>>> through
> >> > > >>>>>>>>>>>>>>> a
> >> > > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate
> from
> >> > > >>>>>>>> storeName1.
> >> > > >>>>>>>>>> It
> >> > > >>>>>>>>>>>>>>> will
> >> > > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update
> the
> >> > > >>>>> store
> >> > > >>>>>>>> and
> >> > > >>>>>>>>>> pass
> >> > > >>>>>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>> new
> >> > > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
> >> > > >>>>>>>> KStreamCogroup
> >> > > >>>>>>>>>>> which
> >> > > >>>>>>>>>>>>>>> is
> >> > > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
> >> > > >> are
> >> > > >>>>> done.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> >> > > >> will
> >> > > >>>>> only
> >> > > >>>>>>>> every
> >> > > >>>>>>>>>>>>>>> create N
> >> > > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> >> > > >>>>>>>>>>>>> matthias@confluent.io
> >> > > >>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> Kyle,
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little
> slow,
> >> > > >>>>> but I
> >> > > >>>>>>>> could
> >> > > >>>>>>>>>> not
> >> > > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> >> > > >>> concrete
> >> > > >>>>>>>> example,
> >> > > >>>>>>>>>>> like
> >> > > >>>>>>>>>>>>>>> 3
> >> > > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected
> result),
> >> > > >>> and
> >> > > >>>>>> show
> >> > > >>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>>>> difference between current way to to implement it
> >> > > >> and
> >> > > >>>>> the
> >> > > >>>>>>>>>> proposed
> >> > > >>>>>>>>>>>>>>> API?
> >> > > >>>>>>>>>>>>>>>>>> This could also cover the internal processing to
> >> > > >> see
> >> > > >>>>> what
> >> > > >>>>>>>> store
> >> > > >>>>>>>>>>> calls
> >> > > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
> >> > > >> and
> >> > > >>> it
> >> > > >>>>>>>> would
> >> > > >>>>>>>>>> help
> >> > > >>>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>>>>> understand it better.
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> Thanks a lot!
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> -Matthias
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> >> > > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found
> here.
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> >> > > >> classes
> >> > > >>>>> and
> >> > > >>>>>> get
> >> > > >>>>>>>>>> around
> >> > > >>>>>>>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> >> > > >> additions.
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> >> > > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> >> > > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> >> > > >>>>>>>>> repartitionIfRequired
> >> > > >>>>>>>>>>>>>>>> method.
> >> > > >>>>>>>>>>>>>>>>> I
> >> > > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
> >> > > >> one
> >> > > >>>>>>>> grouped
> >> > > >>>>>>>>>>> stream
> >> > > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> >> > > >> name
> >> > > >>>>> that
> >> > > >>>>>>>>> someone
> >> > > >>>>>>>>>>>>>>> can
> >> > > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow
> it
> >> > > >> to
> >> > > >>>>> fall
> >> > > >>>>>>>> back
> >> > > >>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped
> tables?
> >> > > >>> This
> >> > > >>>>>>>> would
> >> > > >>>>>>>>> be
> >> > > >>>>>>>>>>>>>>>> pretty
> >> > > >>>>>>>>>>>>>>>>>> easy
> >> > > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
> >> > > >>> stores
> >> > > >>>>>> and
> >> > > >>>>>>>>>>> windowed
> >> > > >>>>>>>>>>>>>>>>>> stores
> >> > > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
> >> > > >>>>>>>>>>>>>>> KTableWindowAggregate
> >> > > >>>>>>>>>>>>>>>>>>> implementations.
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> Thanks,
> >> > > >>>>>>>>>>>>>>>>>>> Kyle
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> >> > > >>>>>>>>> eno.thereska@gmail.com>
> >> > > >>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> >> > > >>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>> Eno
> >> > > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> >> > > >>>>>>>>> damian.guy@gmail.com>
> >> > > >>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> >> > > >>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
> >> > > >> had
> >> > > >>>>> the
> >> > > >>>>>>>>> chance
> >> > > >>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>>> look
> >> > > >>>>>>>>>>>>>>>>>> at
> >> > > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to
> look
> >> > > >>>>> into
> >> > > >>>>>> it
> >> > > >>>>>>>>>>>>>>> tomorrow.
> >> > > >>>>>>>>>>>>>>>>> For
> >> > > >>>>>>>>>>>>>>>>>>>> the
> >> > > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against
> kafka
> >> > > >>>>> trunk
> >> > > >>>>>>>> and
> >> > > >>>>>>>>>> mark
> >> > > >>>>>>>>>>>>>>> it
> >> > > >>>>>>>>>>>>>>>> as
> >> > > >>>>>>>>>>>>>>>>>>>> WIP?
> >> > > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have
> done.
> >> > > >>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> >> > > >>>>>>>>>>>>>>>>>>>>> Damian
> >> > > >>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> >> > > >> some
> >> > > >>>>>>>> attention
> >> > > >>>>>>>>> to
> >> > > >>>>>>>>>>> my
> >> > > >>>>>>>>>>>>>>>> KIP
> >> > > >>>>>>>>>>>>>>>>>> as
> >> > > >>>>>>>>>>>>>>>>>>>> I
> >> > > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> >> > > >> This
> >> > > >>>>> is my
> >> > > >>>>>>>>> first
> >> > > >>>>>>>>>>> KIP
> >> > > >>>>>>>>>>>>>>>> and
> >> > > >>>>>>>>>>>>>>>>>> my
> >> > > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so
> I'm
> >> > > >>>>> sure I
> >> > > >>>>>>>> did
> >> > > >>>>>>>>>>>>>>>> something
> >> > > >>>>>>>>>>>>>>>>>>>> wrong.
> >> > > >>>>>>>>>>>>>>>>>>>>>> ;)
> >> > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> >> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> >> > > >>>>>>>>>>>>>>>>>>>> wrote:
> >> > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> >> > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> >> > > >> discussion
> >> > > >>>>> about
> >> > > >>>>>>>>> adding
> >> > > >>>>>>>>>>>>>>>>> cogroup
> >> > > >>>>>>>>>>>>>>>>>> to
> >> > > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> >> > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> >> > > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >> > > >>>>>> confluence/display/KAFKA/KIP-
> >> > > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> >> > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
> >> > > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> >> > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >> > > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> >> > > >>>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>>>
> >> > > >>>>>>>>>>>
> >> > > >>>>>>>>>>>
> >> > > >>>>>>>>>>
> >> > > >>>>>>>>>
> >> > > >>>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>>
> >> > > >>>>>>> --
> >> > > >>>>>>> -- Guozhang
> >> > > >>>>>>>
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>>
> >> > > >>>>>> --
> >> > > >>>>>> -- Guozhang
> >> > > >>>>>>
> >> > > >>>>>
> >> > > >>>>>
> >> > > >>>>>
> >> > > >>>>> --
> >> > > >>>>> -- Guozhang
> >> > > >>>>>
> >> > > >>>>
> >> > > >>>
> >> > > >>
> >> > >
> >> > >
> >> >
> >> >
> >> > --
> >> > -- Guozhang
> >> >
> >>
> >
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
On a second thought... This is the current proposal API


```

<T> CogroupedKStream<K, T> cogroup(final Initializer<T> initializer, final
Aggregator<? super K, ? super V, T> aggregator, final Serde<T>
aggValueSerde)

```


If we do not have the initializer in the first co-group it might be a bit
awkward for users to specify the aggregator that returns a typed <T> value?
Maybe it is still better to put these two functions in the same api?



Guozhang

On Thu, Jun 8, 2017 at 11:08 AM, Guozhang Wang <wa...@gmail.com> wrote:

> This suggestion lgtm. I would vote for the first alternative than adding
> it to the `KStreamBuilder` though.
>
> On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io>
> wrote:
>
>> I have a minor suggestion to make the API a little bit more symmetric.
>> I feel it would make more sense to move the initializer and serde to the
>> final aggregate statement, since the serde only applies to the state
>> store,
>> and the initializer doesn't bear any relation to the first group in
>> particular. It would end up looking like this:
>>
>> KTable<K, CG> cogrouped =
>>     grouped1.cogroup(aggregator1)
>>             .cogroup(grouped2, aggregator2)
>>             .cogroup(grouped3, aggregator3)
>>             .aggregate(initializer1, aggValueSerde, storeName1);
>>
>> Alternatively, we could move the the first cogroup() method to
>> KStreamBuilder, similar to how we have .merge()
>> and end up with an api that would be even more symmetric.
>>
>> KStreamBuilder.cogroup(grouped1, aggregator1)
>>               .cogroup(grouped2, aggregator2)
>>               .cogroup(grouped3, aggregator3)
>>               .aggregate(initializer1, aggValueSerde, storeName1);
>>
>> This doesn't have to be a blocker, but I thought it would make the API
>> just
>> a tad cleaner.
>>
>> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> wrote:
>>
>> > Kyle,
>> >
>> > Thanks a lot for the updated KIP. It looks good to me.
>> >
>> >
>> > Guozhang
>> >
>> >
>> > On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> wrote:
>> >
>> > > This makes much more sense to me. +1
>> > >
>> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>> winkelman.kyle@gmail.com>
>> > > wrote:
>> > > >
>> > > > I have updated the KIP and my PR. Let me know what you think.
>> > > > To created a cogrouped stream just call cogroup on a KgroupedStream
>> and
>> > > > supply the initializer, aggValueSerde, and an aggregator. Then
>> continue
>> > > > adding kgroupedstreams and aggregators. Then call one of the many
>> > > aggregate
>> > > > calls to create a KTable.
>> > > >
>> > > > Thanks,
>> > > > Kyle
>> > > >
>> > > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:
>> > > >
>> > > >> Hi Kyle,
>> > > >>
>> > > >> Thanks for the update. I think just one initializer makes sense as
>> it
>> > > >> should only be called once per key and generally it is just going
>> to
>> > > create
>> > > >> a new instance of whatever the Aggregate class is.
>> > > >>
>> > > >> Cheers,
>> > > >> Damian
>> > > >>
>> > > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
>> winkelman.kyle@gmail.com
>> > >
>> > > >> wrote:
>> > > >>
>> > > >>> Hello all,
>> > > >>>
>> > > >>> I have spent some more time on this and the best alternative I
>> have
>> > > come
>> > > >> up
>> > > >>> with is:
>> > > >>> KGroupedStream has a single cogroup call that takes an initializer
>> > and
>> > > an
>> > > >>> aggregator.
>> > > >>> CogroupedKStream has a cogroup call that takes additional
>> > groupedStream
>> > > >>> aggregator pairs.
>> > > >>> CogroupedKStream has multiple aggregate methods that create the
>> > > different
>> > > >>> stores.
>> > > >>>
>> > > >>> I plan on updating the kip but I want people's input on if we
>> should
>> > > have
>> > > >>> the initializer be passed in once at the beginning or if we should
>> > > >> instead
>> > > >>> have the initializer be required for each call to one of the
>> > aggregate
>> > > >>> calls. The first makes more sense to me but doesnt allow the user
>> to
>> > > >>> specify different initializers for different tables.
>> > > >>>
>> > > >>> Thanks,
>> > > >>> Kyle
>> > > >>>
>> > > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
>> winkelman.kyle@gmail.com>
>> > > >>> wrote:
>> > > >>>
>> > > >>>> Yea I really like that idea I'll see what I can do to update the
>> kip
>> > > >> and
>> > > >>>> my pr when I have some time. I'm not sure how well creating the
>> > > >>>> kstreamaggregates will go though because at that point I will
>> have
>> > > >> thrown
>> > > >>>> away the type of the values. It will be type safe I just may
>> need to
>> > > >> do a
>> > > >>>> little forcing.
>> > > >>>>
>> > > >>>> Thanks,
>> > > >>>> Kyle
>> > > >>>>
>> > > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com>
>> > wrote:
>> > > >>>>
>> > > >>>>> Kyle,
>> > > >>>>>
>> > > >>>>> Thanks for the explanations, my previous read on the wiki
>> examples
>> > > was
>> > > >>>>> wrong.
>> > > >>>>>
>> > > >>>>> So I guess my motivation should be "reduced" to: can we move the
>> > > >> window
>> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
>> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>> > > >>>>>
>> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream from
>> 3
>> > to
>> > > >> 2.
>> > > >>>>> 2. major: this is for extensibility of the APIs, and since we
>> are
>> > > >>> removing
>> > > >>>>> the "Evolving" annotations on Streams it may be harder to
>> change it
>> > > >>> again
>> > > >>>>> in the future. The extended use cases are that people wanted to
>> > have
>> > > >>>>> windowed running aggregates on different granularities, e.g.
>> "give
>> > me
>> > > >>> the
>> > > >>>>> counts per-minute, per-hour, per-day and per-week", and today in
>> > DSL
>> > > >> we
>> > > >>>>> need to specify that case in multiple aggregate operators, which
>> > gets
>> > > >> a
>> > > >>>>> state store / changelog, etc. And it is possible to optimize it
>> as
>> > > >> well
>> > > >>> to
>> > > >>>>> a single state store. Its implementation would be tricky as you
>> > need
>> > > >> to
>> > > >>>>> contain different lengthed windows within your window store but
>> > just
>> > > >>> from
>> > > >>>>> the public API point of view, it could be specified as:
>> > > >>>>>
>> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>> > > >>>>> "state-store-name");
>> > > >>>>>
>> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
>> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
>> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
>> > > >>>>>
>> > > >>>>> while underlying we are only using a single store
>> > "state-store-name"
>> > > >> for
>> > > >>>>> it.
>> > > >>>>>
>> > > >>>>>
>> > > >>>>> Although this feature is out of the scope of this KIP, I'd like
>> to
>> > > >>> discuss
>> > > >>>>> if we can "leave the door open" to make such changes without
>> > > modifying
>> > > >>> the
>> > > >>>>> public APIs .
>> > > >>>>>
>> > > >>>>> Guozhang
>> > > >>>>>
>> > > >>>>>
>> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>> > > >>> winkelman.kyle@gmail.com
>> > > >>>>>>
>> > > >>>>> wrote:
>> > > >>>>>
>> > > >>>>>> I allow defining a single window/sessionwindow one time when
>> you
>> > > >> make
>> > > >>>>> the
>> > > >>>>>> cogroup call from a KGroupedStream. From then on you are using
>> the
>> > > >>>>> cogroup
>> > > >>>>>> call from with in CogroupedKStream which doesnt accept any
>> > > >> additional
>> > > >>>>>> windows/sessionwindows.
>> > > >>>>>>
>> > > >>>>>> Is this what you meant by your question or did I misunderstand?
>> > > >>>>>>
>> > > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
>> > > >> wrote:
>> > > >>>>>>
>> > > >>>>>> Another question that came to me is on "window alignment": from
>> > the
>> > > >>> KIP
>> > > >>>>> it
>> > > >>>>>> seems you are allowing users to specify a (potentially
>> different)
>> > > >>> window
>> > > >>>>>> spec in each co-grouped input stream. So if these window specs
>> are
>> > > >>>>>> different how should we "align" them with different input
>> > streams? I
>> > > >>>>> think
>> > > >>>>>> it is more natural to only specify on window spec in the
>> > > >>>>>>
>> > > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> And remove it from the cogroup() functions. WDYT?
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> Guozhang
>> > > >>>>>>
>> > > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
>> > wangguoz@gmail.com>
>> > > >>>>> wrote:
>> > > >>>>>>
>> > > >>>>>>> Thanks for the proposal Kyle, this is a quite common use case
>> to
>> > > >>>>> support
>> > > >>>>>>> such multi-way table join (i.e. N source tables with N
>> aggregate
>> > > >>> func)
>> > > >>>>>> with
>> > > >>>>>>> a single store and N+1 serdes, I have seen lots of people
>> using
>> > > >> the
>> > > >>>>>>> low-level PAPI to achieve this goal.
>> > > >>>>>>>
>> > > >>>>>>>
>> > > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>> > > >>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>> wrote:
>> > > >>>>>>>
>> > > >>>>>>>> I like your point about not handling other cases such as
>> count
>> > > >> and
>> > > >>>>>> reduce.
>> > > >>>>>>>>
>> > > >>>>>>>> I think that reduce may not make sense because reduce assumes
>> > > >> that
>> > > >>>>> the
>> > > >>>>>>>> input values are the same as the output values. With cogroup
>> > > >> there
>> > > >>>>> may
>> > > >>>>>> be
>> > > >>>>>>>> multiple different input types and then your output type
>> cant be
>> > > >>>>>> multiple
>> > > >>>>>>>> different things. In the case where you have all matching
>> value
>> > > >>> types
>> > > >>>>>> you
>> > > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
>> > > >>>>>>>>
>> > > >>>>>>>> As for count I think it is possible to call count on all the
>> > > >>>>> individual
>> > > >>>>>>>> grouped streams and then do joins. Otherwise we could maybe
>> make
>> > > >> a
>> > > >>>>>> special
>> > > >>>>>>>> call in groupedstream for this case. Because in this case we
>> > dont
>> > > >>>>> need
>> > > >>>>>> to
>> > > >>>>>>>> do type checking on the values. It could be similar to the
>> > > >> current
>> > > >>>>> count
>> > > >>>>>>>> methods but accept a var args of additonal grouped streams as
>> > > >> well
>> > > >>>>> and
>> > > >>>>>>>> make
>> > > >>>>>>>> sure they have a key type of K.
>> > > >>>>>>>>
>> > > >>>>>>>> The way I have put the kip together is to ensure that we do
>> type
>> > > >>>>>> checking.
>> > > >>>>>>>> I don't see a way we could group them all first and then
>> make a
>> > > >>> call
>> > > >>>>> to
>> > > >>>>>>>> count, reduce, or aggregate because with aggregate they would
>> > > >> need
>> > > >>> to
>> > > >>>>>> pass
>> > > >>>>>>>> a list of aggregators and we would have no way of type
>> checking
>> > > >>> that
>> > > >>>>>> they
>> > > >>>>>>>> match the grouped streams.
>> > > >>>>>>>>
>> > > >>>>>>>> Thanks,
>> > > >>>>>>>> Kyle
>> > > >>>>>>>>
>> > > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
>> xavier@confluent.io>
>> > > >>>>> wrote:
>> > > >>>>>>>>
>> > > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a very
>> > > >>> useful
>> > > >>>>>>>>> addition and wanted to provide an additional use-case and
>> some
>> > > >>> more
>> > > >>>>>>>>> comments.
>> > > >>>>>>>>>
>> > > >>>>>>>>> This is actually a very common analytics use-case in the
>> > > >> ad-tech
>> > > >>>>>>>> industry.
>> > > >>>>>>>>> The typical setup will have an auction stream, an impression
>> > > >>>>> stream,
>> > > >>>>>>>> and a
>> > > >>>>>>>>> click stream. Those three streams need to be combined to
>> > > >> compute
>> > > >>>>>>>> aggregate
>> > > >>>>>>>>> statistics (e.g. impression statistics, and click-through
>> > > >> rates),
>> > > >>>>>> since
>> > > >>>>>>>>> most of the attributes of interest are only present the
>> auction
>> > > >>>>>> stream.
>> > > >>>>>>>>>
>> > > >>>>>>>>> A simple way to do this is to co-group all the streams by
>> the
>> > > >>>>> auction
>> > > >>>>>>>> key,
>> > > >>>>>>>>> and process updates to the co-group as events for each
>> stream
>> > > >>> come
>> > > >>>>> in,
>> > > >>>>>>>>> keeping only one value from each stream before sending
>> > > >> downstream
>> > > >>>>> for
>> > > >>>>>>>>> further processing / aggregation.
>> > > >>>>>>>>>
>> > > >>>>>>>>> One could view the result of that co-group operation as a
>> > > >>> "KTable"
>> > > >>>>>> with
>> > > >>>>>>>>> multiple values per key. The key being the grouping key, and
>> > > >> the
>> > > >>>>>> values
>> > > >>>>>>>>> consisting of one value per stream.
>> > > >>>>>>>>>
>> > > >>>>>>>>> What I like about Kyle's approach is that allows elegant
>> > > >>>>> co-grouping
>> > > >>>>>> of
>> > > >>>>>>>>> multiple streams without having to worry about the number of
>> > > >>>>> streams,
>> > > >>>>>>>> and
>> > > >>>>>>>>> avoids dealing with Tuple types or other generic interfaces
>> > > >> that
>> > > >>>>> could
>> > > >>>>>>>> get
>> > > >>>>>>>>> messy if we wanted to preserve all the value types in the
>> > > >>> resulting
>> > > >>>>>>>>> co-grouped stream.
>> > > >>>>>>>>>
>> > > >>>>>>>>> My only concern is that we only allow the cogroup +
>> aggregate
>> > > >>>>> combined
>> > > >>>>>>>>> operation. This forces the user to build their own tuple
>> > > >>>>> serialization
>> > > >>>>>>>>> format if they want to preserve the individual input stream
>> > > >>> values
>> > > >>>>> as
>> > > >>>>>> a
>> > > >>>>>>>>> group. It also deviates quite a bit from our approach in
>> > > >>>>>> KGroupedStream
>> > > >>>>>>>>> which offers other operations, such as count and reduce,
>> which
>> > > >>>>> should
>> > > >>>>>>>> also
>> > > >>>>>>>>> be applicable to a co-grouped stream.
>> > > >>>>>>>>>
>> > > >>>>>>>>> Overall I still think this is a really useful addition, but
>> I
>> > > >>> feel
>> > > >>>>> we
>> > > >>>>>>>>> haven't spend much time trying to explore alternative DSLs
>> that
>> > > >>>>> could
>> > > >>>>>>>> maybe
>> > > >>>>>>>>> generalize better or match our existing syntax more closely.
>> > > >>>>>>>>>
>> > > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>> > > >>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>>>
>> > > >>>>>>>>> wrote:
>> > > >>>>>>>>>
>> > > >>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
>> > > >>> streams
>> > > >>>>>> realm
>> > > >>>>>>>>> that
>> > > >>>>>>>>>> I should reach out to for input?
>> > > >>>>>>>>>>
>> > > >>>>>>>>>> I believe Damian Guy is still planning on reviewing this
>> more
>> > > >>> in
>> > > >>>>>> depth
>> > > >>>>>>>>> so I
>> > > >>>>>>>>>> will wait for his inputs before continuing.
>> > > >>>>>>>>>>
>> > > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
>> > > >> eno.thereska@gmail.com
>> > > >>>>
>> > > >>>>>>>> wrote:
>> > > >>>>>>>>>>
>> > > >>>>>>>>>>> Thanks Kyle, good arguments.
>> > > >>>>>>>>>>>
>> > > >>>>>>>>>>> Eno
>> > > >>>>>>>>>>>
>> > > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>> > > >>>>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>>>>
>> > > >>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> *- minor: could you add an exact example (similar to what
>> > > >>>>> Jay’s
>> > > >>>>>>>>> example
>> > > >>>>>>>>>>> is,
>> > > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
>> > > >>>>>> concrete?*
>> > > >>>>>>>>>>>> I have added a more concrete example to the KIP.
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> *- my main concern is that we’re exposing this
>> > > >> optimization
>> > > >>>>> to
>> > > >>>>>> the
>> > > >>>>>>>>> DSL.
>> > > >>>>>>>>>>> In
>> > > >>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
>> > > >>> and
>> > > >>>>> do
>> > > >>>>>>>> the
>> > > >>>>>>>>>> right
>> > > >>>>>>>>>>>> thing under the covers (create just one state store,
>> > > >>> arrange
>> > > >>>>> the
>> > > >>>>>>>>> nodes
>> > > >>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
>> > > >>>>> pieces
>> > > >>>>>>>>> (group,
>> > > >>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
>> > > >>>>> like to
>> > > >>>>>>>> hear
>> > > >>>>>>>>>>> your
>> > > >>>>>>>>>>>> thoughts on whether it’s possible to do this optimization
>> > > >>>>> with
>> > > >>>>>> the
>> > > >>>>>>>>>>> current
>> > > >>>>>>>>>>>> DSL, at the topology builder level.*
>> > > >>>>>>>>>>>> You would have to make a lot of checks to understand if
>> > > >> it
>> > > >>> is
>> > > >>>>>> even
>> > > >>>>>>>>>>> possible
>> > > >>>>>>>>>>>> to make this optimization:
>> > > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
>> > > >>>>>>>>>>>> 2. None of the intermediate KTables are used for anything
>> > > >>>>> else.
>> > > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This may be
>> > > >>>>>>>> impossible
>> > > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
>> > > >>> topology
>> > > >>>>> has
>> > > >>>>>>>>>> already
>> > > >>>>>>>>>>>> been built.)
>> > > >>>>>>>>>>>> You would then need to make decisions during the
>> > > >>>>> optimization:
>> > > >>>>>>>>>>>> 1. Your new initializer would the composite of all the
>> > > >>>>>> individual
>> > > >>>>>>>>>>>> initializers and the valueJoiners.
>> > > >>>>>>>>>>>> 2. I am having a hard time thinking about how you would
>> > > >>> turn
>> > > >>>>> the
>> > > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
>> > > >> would
>> > > >>>>> work
>> > > >>>>>> on
>> > > >>>>>>>>> the
>> > > >>>>>>>>>>>> final object, but this may be possible.
>> > > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
>> > > >> would
>> > > >>>>> be
>> > > >>>>>> for
>> > > >>>>>>>>> the
>> > > >>>>>>>>>>>> aggregate values. None of the declared ones would be
>> > > >>>>> guaranteed
>> > > >>>>>> to
>> > > >>>>>>>>> hold
>> > > >>>>>>>>>>> the
>> > > >>>>>>>>>>>> final object. This would mean you must created a new
>> > > >> state
>> > > >>>>> store
>> > > >>>>>>>> and
>> > > >>>>>>>>>> not
>> > > >>>>>>>>>>>> created any of the declared ones.
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> The main argument I have against it is even if it could
>> > > >> be
>> > > >>>>> done
>> > > >>>>>> I
>> > > >>>>>>>>> don't
>> > > >>>>>>>>>>>> know that we would want to have this be an optimization
>> > > >> in
>> > > >>>>> the
>> > > >>>>>>>>>> background
>> > > >>>>>>>>>>>> because the user would still be required to think about
>> > > >> all
>> > > >>>>> of
>> > > >>>>>> the
>> > > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
>> > > >> about
>> > > >>>>> if
>> > > >>>>>>>> they
>> > > >>>>>>>>>> only
>> > > >>>>>>>>>>>> care about the final object.
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
>> > > >>> should
>> > > >>>>> be
>> > > >>>>>>>> part
>> > > >>>>>>>>> of
>> > > >>>>>>>>>>> the
>> > > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
>> > > >> want
>> > > >>> to
>> > > >>>>>>>> allow
>> > > >>>>>>>>>>> people
>> > > >>>>>>>>>>>> to join more than 2 or more streams in an easy way. Right
>> > > >>>>> now I
>> > > >>>>>>>> don't
>> > > >>>>>>>>>>> think
>> > > >>>>>>>>>>>> we give them ways of handling this use case easily.
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> *-I think there will be scope for several such
>> > > >>> optimizations
>> > > >>>>> in
>> > > >>>>>>>> the
>> > > >>>>>>>>>>> future
>> > > >>>>>>>>>>>> and perhaps at some point we need to think about
>> > > >> decoupling
>> > > >>>>> the
>> > > >>>>>>>> 1:1
>> > > >>>>>>>>>>> mapping
>> > > >>>>>>>>>>>> from the DSL into the physical topology.*
>> > > >>>>>>>>>>>> I would argue that cogroup is not just an optimization it
>> > > >>> is
>> > > >>>>> a
>> > > >>>>>> new
>> > > >>>>>>>>> way
>> > > >>>>>>>>>>> for
>> > > >>>>>>>>>>>> the users to look at accomplishing a problem that
>> > > >> requires
>> > > >>>>>>>> multiple
>> > > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
>> > > >> think
>> > > >>>>>> users
>> > > >>>>>>>>>> should
>> > > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
>> > > >>> their
>> > > >>>>>>>>>>> initializers,
>> > > >>>>>>>>>>>> serdes and stores if all they care about is the final
>> > > >>> object.
>> > > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
>> > > >> supply
>> > > >>>>>>>> additional
>> > > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
>> > > >> single
>> > > >>>>>>>> grouped
>> > > >>>>>>>>>>> stream
>> > > >>>>>>>>>>>> making an aggregate call. This case is what I view an
>> > > >>>>>> optimization
>> > > >>>>>>>>> as,
>> > > >>>>>>>>>> we
>> > > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
>> > > >>> just
>> > > >>>>> a
>> > > >>>>>>>> call
>> > > >>>>>>>>> to
>> > > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
>> > > >>>>>>>> KGroupedStream#cogroup.
>> > > >>>>>>>>> (I
>> > > >>>>>>>>>>>> would prefer to just write a warning saying that this is
>> > > >>> not
>> > > >>>>> how
>> > > >>>>>>>>>> cogroup
>> > > >>>>>>>>>>> is
>> > > >>>>>>>>>>>> to be used.)
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> Thanks,
>> > > >>>>>>>>>>>> Kyle
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
>> > > >>>>>>>> eno.thereska@gmail.com
>> > > >>>>>>>>>>
>> > > >>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>>> Hi Kyle,
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>> - minor: could you add an exact example (similar to what
>> > > >>>>> Jay’s
>> > > >>>>>>>>> example
>> > > >>>>>>>>>>> is,
>> > > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
>> > > >>>>>> concrete?
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>> - my main concern is that we’re exposing this
>> > > >> optimization
>> > > >>>>> to
>> > > >>>>>> the
>> > > >>>>>>>>> DSL.
>> > > >>>>>>>>>>> In
>> > > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
>> > > >>>>> and do
>> > > >>>>>>>> the
>> > > >>>>>>>>>>> right
>> > > >>>>>>>>>>>>> thing under the covers (create just one state store,
>> > > >>> arrange
>> > > >>>>>> the
>> > > >>>>>>>>> nodes
>> > > >>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
>> > > >>>>> pieces
>> > > >>>>>>>>> (group,
>> > > >>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
>> > > >>>>> like
>> > > >>>>>> to
>> > > >>>>>>>>> hear
>> > > >>>>>>>>>>> your
>> > > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
>> > > >> optimization
>> > > >>>>> with
>> > > >>>>>>>> the
>> > > >>>>>>>>>>> current
>> > > >>>>>>>>>>>>> DSL, at the topology builder level.
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>> I think there will be scope for several such
>> > > >> optimizations
>> > > >>>>> in
>> > > >>>>>> the
>> > > >>>>>>>>>> future
>> > > >>>>>>>>>>>>> and perhaps at some point we need to think about
>> > > >>> decoupling
>> > > >>>>> the
>> > > >>>>>>>> 1:1
>> > > >>>>>>>>>>> mapping
>> > > >>>>>>>>>>>>> from the DSL into the physical topology.
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>> Thanks
>> > > >>>>>>>>>>>>> Eno
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
>> > > >> jay@confluent.io>
>> > > >>>>>> wrote:
>> > > >>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
>> > > >>> pretty
>> > > >>>>>>>> common.
>> > > >>>>>>>>> An
>> > > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
>> > > >> customer
>> > > >>>>>>>> profile"
>> > > >>>>>>>>> use
>> > > >>>>>>>>>>>>> case
>> > > >>>>>>>>>>>>>> we often use. In that use case you have a dozen systems
>> > > >>>>> each
>> > > >>>>>> of
>> > > >>>>>>>>> which
>> > > >>>>>>>>>>> has
>> > > >>>>>>>>>>>>>> some information about your customer (account details,
>> > > >>>>>> settings,
>> > > >>>>>>>>>>> billing
>> > > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
>> > > >> etc).
>> > > >>>>> Your
>> > > >>>>>>>> goal
>> > > >>>>>>>>> is
>> > > >>>>>>>>>>> to
>> > > >>>>>>>>>>>>>> join/munge these into a single profile record for each
>> > > >>>>>> customer
>> > > >>>>>>>>> that
>> > > >>>>>>>>>>> has
>> > > >>>>>>>>>>>>>> all the relevant info in a usable form and is
>> > > >> up-to-date
>> > > >>>>> with
>> > > >>>>>>>> all
>> > > >>>>>>>>> the
>> > > >>>>>>>>>>>>>> source systems. If you implement that with kstreams as
>> > > >> a
>> > > >>>>>>>> sequence
>> > > >>>>>>>>> of
>> > > >>>>>>>>>>>>> joins
>> > > >>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
>> > > >>>>> tables.
>> > > >>>>>>>> But
>> > > >>>>>>>>>>> clearly
>> > > >>>>>>>>>>>>>> you only need a single stage to group all these things
>> > > >>> that
>> > > >>>>>> are
>> > > >>>>>>>>>> already
>> > > >>>>>>>>>>>>>> co-partitioned. A distributed database would do this
>> > > >>> under
>> > > >>>>> the
>> > > >>>>>>>>> covers
>> > > >>>>>>>>>>>>> which
>> > > >>>>>>>>>>>>>> is arguably better (at least when it does the right
>> > > >>> thing)
>> > > >>>>> and
>> > > >>>>>>>>>> perhaps
>> > > >>>>>>>>>>> we
>> > > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
>> > > >>>>>>>> partitioning
>> > > >>>>>>>>> so
>> > > >>>>>>>>>> we
>> > > >>>>>>>>>>>>> may
>> > > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they are
>> > > >>>>>> already
>> > > >>>>>>>>>>>>>> co-partitioned.
>> > > >>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>> -Jay
>> > > >>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
>> > > >>>>>>>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
>> > > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
>> > > >> other
>> > > >>>>>>>> platforms
>> > > >>>>>>>>>>> like
>> > > >>>>>>>>>>>>>>> spark and pig.
>> > > >>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
>> > > >>>>>>>>>>>>> PairRDDFunctions.html
>> > > >>>>>>>>>>>>>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
>> > > >>>>>>>>>>>>> cogroup_operator.htm
>> > > >>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
>> > > >>>>> damian.guy@gmail.com>
>> > > >>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>> Hi Kyle,
>> > > >>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
>> > > >> outer
>> > > >>>>>> join?
>> > > >>>>>>>> So
>> > > >>>>>>>>> an
>> > > >>>>>>>>>>>>> input
>> > > >>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
>> > > >>> value
>> > > >>>>> -
>> > > >>>>>> is
>> > > >>>>>>>>> that
>> > > >>>>>>>>>>>>>>> correct?
>> > > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
>> > > >> current
>> > > >>>>>> value,
>> > > >>>>>>>>>>>>> aggregates
>> > > >>>>>>>>>>>>>>>> and forwards the result.
>> > > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit more,
>> > > >>>>> but it
>> > > >>>>>>>>> seems
>> > > >>>>>>>>>>> like
>> > > >>>>>>>>>>>>>>> it
>> > > >>>>>>>>>>>>>>>> could be a useful optimization.
>> > > >>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
>> > > >>>>>>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> I sure can. I have added the following description
>> > > >> to
>> > > >>> my
>> > > >>>>>>>> KIP. If
>> > > >>>>>>>>>>> this
>> > > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
>> > > >>> time
>> > > >>>>> to
>> > > >>>>>>>>> build a
>> > > >>>>>>>>>>>>>>>> diagram
>> > > >>>>>>>>>>>>>>>>> and make more of a step by step description:
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> Example with Current API:
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
>> > > >>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
>> > > >>>>>> initializer1
>> > > >>>>>>>> ,
>> > > >>>>>>>>>>>>>>>> aggregator1,
>> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
>> > > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
>> > > >>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
>> > > >>>>>> initializer2
>> > > >>>>>>>> ,
>> > > >>>>>>>>>>>>>>>> aggregator2,
>> > > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
>> > > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
>> > > >>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
>> > > >>>>>> initializer3
>> > > >>>>>>>> ,
>> > > >>>>>>>>>>>>>>>> aggregator3,
>> > > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
>> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
>> > > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
>> > > >>>>> joinerOneTwoAndThree);
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores, requires
>> > > >> 3
>> > > >>>>>>>>>> initializers,
>> > > >>>>>>>>>>>>>>> and 3
>> > > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to user
>> > > >> to
>> > > >>>>>> define
>> > > >>>>>>>>> what
>> > > >>>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
>> > > >> They
>> > > >>>>> are
>> > > >>>>>>>> left
>> > > >>>>>>>>>>> with a
>> > > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all the
>> > > >>>>> same
>> > > >>>>>> as
>> > > >>>>>>>> CG
>> > > >>>>>>>>>> and
>> > > >>>>>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>> two
>> > > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
>> > > >>>>>>>> intermediate
>> > > >>>>>>>>>>> states
>> > > >>>>>>>>>>>>>>>> such
>> > > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
>> > > >> joiners
>> > > >>>>> use
>> > > >>>>>>>> those
>> > > >>>>>>>>>> to
>> > > >>>>>>>>>>>>>>> build
>> > > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something the
>> > > >>> user
>> > > >>>>>>>> could
>> > > >>>>>>>>>> avoid
>> > > >>>>>>>>>>>>>>>>> thinking about with this KIP.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>> > > >> will
>> > > >>>>> first
>> > > >>>>>>>> go
>> > > >>>>>>>>>>> through
>> > > >>>>>>>>>>>>>>> a
>> > > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate from
>> > > >>>>>>>> storeName1.
>> > > >>>>>>>>>> It
>> > > >>>>>>>>>>>>>>> will
>> > > >>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
>> > > >>> value
>> > > >>>>>> and
>> > > >>>>>>>> get
>> > > >>>>>>>>>>> sent
>> > > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
>> > > >> up
>> > > >>>>> the
>> > > >>>>>>>>> current
>> > > >>>>>>>>>>>>> value
>> > > >>>>>>>>>>>>>>>> of
>> > > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first joiner
>> > > >> to
>> > > >>>>>>>> calculate
>> > > >>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>> second
>> > > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
>> > > >>> additional
>> > > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
>> > > >>> current
>> > > >>>>>>>> value of
>> > > >>>>>>>>>> the
>> > > >>>>>>>>>>>>>>> key
>> > > >>>>>>>>>>>>>>>> in
>> > > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
>> > > >>> final
>> > > >>>>>>>>> aggregate
>> > > >>>>>>>>>>>>>>> value.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> If you think through all possibilities for incoming
>> > > >>>>> topics
>> > > >>>>>>>> you
>> > > >>>>>>>>>> will
>> > > >>>>>>>>>>>>> see
>> > > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
>> > > >>> three
>> > > >>>>>>>> stores
>> > > >>>>>>>>>> are
>> > > >>>>>>>>>>>>>>>> queried
>> > > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates N
>> > > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
>> > > >> and
>> > > >>>>> N-1
>> > > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> Example with Proposed API:
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
>> > > >>>>> builder.stream("topic1").
>> > > >>>>>>>>>>>>>>> groupByKey();
>> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
>> > > >>>>> builder.stream("topic2").
>> > > >>>>>>>>>>>>>>> groupByKey();
>> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
>> > > >>>>> builder.stream("topic3").
>> > > >>>>>>>>>>>>>>> groupByKey();
>> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
>> > > >>> grouped1.cogroup(initializer1,
>> > > >>>>>>>>>>> aggregator1,
>> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
>> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
>> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
>> > > >>>>>>>>>>>>>>>>>      .aggregate();
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore, requires 1
>> > > >>>>>>>>> initializer,
>> > > >>>>>>>>>>> and
>> > > >>>>>>>>>>>>> 1
>> > > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry about
>> > > >>> the
>> > > >>>>>>>>>>> intermediate
>> > > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think about
>> > > >>> is
>> > > >>>>> how
>> > > >>>>>>>> each
>> > > >>>>>>>>>>>>> stream
>> > > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>> > > >> will
>> > > >>>>> first
>> > > >>>>>>>> go
>> > > >>>>>>>>>>> through
>> > > >>>>>>>>>>>>>>> a
>> > > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate from
>> > > >>>>>>>> storeName1.
>> > > >>>>>>>>>> It
>> > > >>>>>>>>>>>>>>> will
>> > > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update the
>> > > >>>>> store
>> > > >>>>>>>> and
>> > > >>>>>>>>>> pass
>> > > >>>>>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>> new
>> > > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
>> > > >>>>>>>> KStreamCogroup
>> > > >>>>>>>>>>> which
>> > > >>>>>>>>>>>>>>> is
>> > > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
>> > > >> are
>> > > >>>>> done.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
>> > > >> will
>> > > >>>>> only
>> > > >>>>>>>> every
>> > > >>>>>>>>>>>>>>> create N
>> > > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
>> > > >>>>>>>>>>>>> matthias@confluent.io
>> > > >>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> Kyle,
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
>> > > >>>>> but I
>> > > >>>>>>>> could
>> > > >>>>>>>>>> not
>> > > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
>> > > >>> concrete
>> > > >>>>>>>> example,
>> > > >>>>>>>>>>> like
>> > > >>>>>>>>>>>>>>> 3
>> > > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected result),
>> > > >>> and
>> > > >>>>>> show
>> > > >>>>>>>>> the
>> > > >>>>>>>>>>>>>>>>>> difference between current way to to implement it
>> > > >> and
>> > > >>>>> the
>> > > >>>>>>>>>> proposed
>> > > >>>>>>>>>>>>>>> API?
>> > > >>>>>>>>>>>>>>>>>> This could also cover the internal processing to
>> > > >> see
>> > > >>>>> what
>> > > >>>>>>>> store
>> > > >>>>>>>>>>> calls
>> > > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
>> > > >> and
>> > > >>> it
>> > > >>>>>>>> would
>> > > >>>>>>>>>> help
>> > > >>>>>>>>>>> to
>> > > >>>>>>>>>>>>>>>>>> understand it better.
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> Thanks a lot!
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> -Matthias
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
>> > > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found here.
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
>> > > >> classes
>> > > >>>>> and
>> > > >>>>>> get
>> > > >>>>>>>>>> around
>> > > >>>>>>>>>>>>>>> to
>> > > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
>> > > >> additions.
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
>> > > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
>> > > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
>> > > >>>>>>>>> repartitionIfRequired
>> > > >>>>>>>>>>>>>>>> method.
>> > > >>>>>>>>>>>>>>>>> I
>> > > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
>> > > >> one
>> > > >>>>>>>> grouped
>> > > >>>>>>>>>>> stream
>> > > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
>> > > >> name
>> > > >>>>> that
>> > > >>>>>>>>> someone
>> > > >>>>>>>>>>>>>>> can
>> > > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow it
>> > > >> to
>> > > >>>>> fall
>> > > >>>>>>>> back
>> > > >>>>>>>>>> to
>> > > >>>>>>>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped tables?
>> > > >>> This
>> > > >>>>>>>> would
>> > > >>>>>>>>> be
>> > > >>>>>>>>>>>>>>>> pretty
>> > > >>>>>>>>>>>>>>>>>> easy
>> > > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
>> > > >>> stores
>> > > >>>>>> and
>> > > >>>>>>>>>>> windowed
>> > > >>>>>>>>>>>>>>>>>> stores
>> > > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
>> > > >>>>>>>>>>>>>>> KTableWindowAggregate
>> > > >>>>>>>>>>>>>>>>>>> implementations.
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> Thanks,
>> > > >>>>>>>>>>>>>>>>>>> Kyle
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
>> > > >>>>>>>>> eno.thereska@gmail.com>
>> > > >>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
>> > > >>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>> Eno
>> > > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
>> > > >>>>>>>>> damian.guy@gmail.com>
>> > > >>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
>> > > >>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
>> > > >> had
>> > > >>>>> the
>> > > >>>>>>>>> chance
>> > > >>>>>>>>>> to
>> > > >>>>>>>>>>>>>>>> look
>> > > >>>>>>>>>>>>>>>>>> at
>> > > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to look
>> > > >>>>> into
>> > > >>>>>> it
>> > > >>>>>>>>>>>>>>> tomorrow.
>> > > >>>>>>>>>>>>>>>>> For
>> > > >>>>>>>>>>>>>>>>>>>> the
>> > > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against kafka
>> > > >>>>> trunk
>> > > >>>>>>>> and
>> > > >>>>>>>>>> mark
>> > > >>>>>>>>>>>>>>> it
>> > > >>>>>>>>>>>>>>>> as
>> > > >>>>>>>>>>>>>>>>>>>> WIP?
>> > > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have done.
>> > > >>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
>> > > >>>>>>>>>>>>>>>>>>>>> Damian
>> > > >>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
>> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
>> > > >> some
>> > > >>>>>>>> attention
>> > > >>>>>>>>> to
>> > > >>>>>>>>>>> my
>> > > >>>>>>>>>>>>>>>> KIP
>> > > >>>>>>>>>>>>>>>>>> as
>> > > >>>>>>>>>>>>>>>>>>>> I
>> > > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
>> > > >> This
>> > > >>>>> is my
>> > > >>>>>>>>> first
>> > > >>>>>>>>>>> KIP
>> > > >>>>>>>>>>>>>>>> and
>> > > >>>>>>>>>>>>>>>>>> my
>> > > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so I'm
>> > > >>>>> sure I
>> > > >>>>>>>> did
>> > > >>>>>>>>>>>>>>>> something
>> > > >>>>>>>>>>>>>>>>>>>> wrong.
>> > > >>>>>>>>>>>>>>>>>>>>>> ;)
>> > > >>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
>> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
>> > > >>>>>>>>>>>>>>>>>>>> wrote:
>> > > >>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
>> > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
>> > > >> discussion
>> > > >>>>> about
>> > > >>>>>>>>> adding
>> > > >>>>>>>>>>>>>>>>> cogroup
>> > > >>>>>>>>>>>>>>>>>> to
>> > > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
>> > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
>> > > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>> > > >>>>>> confluence/display/KAFKA/KIP-
>> > > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
>> > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
>> > > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
>> > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
>> > > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
>> > > >>>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>>>
>> > > >>>>>>>>>>>
>> > > >>>>>>>>>>>
>> > > >>>>>>>>>>
>> > > >>>>>>>>>
>> > > >>>>>>>>
>> > > >>>>>>>
>> > > >>>>>>>
>> > > >>>>>>>
>> > > >>>>>>> --
>> > > >>>>>>> -- Guozhang
>> > > >>>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>>
>> > > >>>>>> --
>> > > >>>>>> -- Guozhang
>> > > >>>>>>
>> > > >>>>>
>> > > >>>>>
>> > > >>>>>
>> > > >>>>> --
>> > > >>>>> -- Guozhang
>> > > >>>>>
>> > > >>>>
>> > > >>>
>> > > >>
>> > >
>> > >
>> >
>> >
>> > --
>> > -- Guozhang
>> >
>>
>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Guozhang Wang <wa...@gmail.com>.
This suggestion lgtm. I would vote for the first alternative than adding it
to the `KStreamBuilder` though.

On Thu, Jun 8, 2017 at 10:58 AM, Xavier Léauté <xa...@confluent.io> wrote:

> I have a minor suggestion to make the API a little bit more symmetric.
> I feel it would make more sense to move the initializer and serde to the
> final aggregate statement, since the serde only applies to the state store,
> and the initializer doesn't bear any relation to the first group in
> particular. It would end up looking like this:
>
> KTable<K, CG> cogrouped =
>     grouped1.cogroup(aggregator1)
>             .cogroup(grouped2, aggregator2)
>             .cogroup(grouped3, aggregator3)
>             .aggregate(initializer1, aggValueSerde, storeName1);
>
> Alternatively, we could move the the first cogroup() method to
> KStreamBuilder, similar to how we have .merge()
> and end up with an api that would be even more symmetric.
>
> KStreamBuilder.cogroup(grouped1, aggregator1)
>               .cogroup(grouped2, aggregator2)
>               .cogroup(grouped3, aggregator3)
>               .aggregate(initializer1, aggValueSerde, storeName1);
>
> This doesn't have to be a blocker, but I thought it would make the API just
> a tad cleaner.
>
> On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Kyle,
> >
> > Thanks a lot for the updated KIP. It looks good to me.
> >
> >
> > Guozhang
> >
> >
> > On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> wrote:
> >
> > > This makes much more sense to me. +1
> > >
> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> winkelman.kyle@gmail.com>
> > > wrote:
> > > >
> > > > I have updated the KIP and my PR. Let me know what you think.
> > > > To created a cogrouped stream just call cogroup on a KgroupedStream
> and
> > > > supply the initializer, aggValueSerde, and an aggregator. Then
> continue
> > > > adding kgroupedstreams and aggregators. Then call one of the many
> > > aggregate
> > > > calls to create a KTable.
> > > >
> > > > Thanks,
> > > > Kyle
> > > >
> > > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:
> > > >
> > > >> Hi Kyle,
> > > >>
> > > >> Thanks for the update. I think just one initializer makes sense as
> it
> > > >> should only be called once per key and generally it is just going to
> > > create
> > > >> a new instance of whatever the Aggregate class is.
> > > >>
> > > >> Cheers,
> > > >> Damian
> > > >>
> > > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <
> winkelman.kyle@gmail.com
> > >
> > > >> wrote:
> > > >>
> > > >>> Hello all,
> > > >>>
> > > >>> I have spent some more time on this and the best alternative I have
> > > come
> > > >> up
> > > >>> with is:
> > > >>> KGroupedStream has a single cogroup call that takes an initializer
> > and
> > > an
> > > >>> aggregator.
> > > >>> CogroupedKStream has a cogroup call that takes additional
> > groupedStream
> > > >>> aggregator pairs.
> > > >>> CogroupedKStream has multiple aggregate methods that create the
> > > different
> > > >>> stores.
> > > >>>
> > > >>> I plan on updating the kip but I want people's input on if we
> should
> > > have
> > > >>> the initializer be passed in once at the beginning or if we should
> > > >> instead
> > > >>> have the initializer be required for each call to one of the
> > aggregate
> > > >>> calls. The first makes more sense to me but doesnt allow the user
> to
> > > >>> specify different initializers for different tables.
> > > >>>
> > > >>> Thanks,
> > > >>> Kyle
> > > >>>
> > > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <
> winkelman.kyle@gmail.com>
> > > >>> wrote:
> > > >>>
> > > >>>> Yea I really like that idea I'll see what I can do to update the
> kip
> > > >> and
> > > >>>> my pr when I have some time. I'm not sure how well creating the
> > > >>>> kstreamaggregates will go though because at that point I will have
> > > >> thrown
> > > >>>> away the type of the values. It will be type safe I just may need
> to
> > > >> do a
> > > >>>> little forcing.
> > > >>>>
> > > >>>> Thanks,
> > > >>>> Kyle
> > > >>>>
> > > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com>
> > wrote:
> > > >>>>
> > > >>>>> Kyle,
> > > >>>>>
> > > >>>>> Thanks for the explanations, my previous read on the wiki
> examples
> > > was
> > > >>>>> wrong.
> > > >>>>>
> > > >>>>> So I guess my motivation should be "reduced" to: can we move the
> > > >> window
> > > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > > >>>>>
> > > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream from 3
> > to
> > > >> 2.
> > > >>>>> 2. major: this is for extensibility of the APIs, and since we are
> > > >>> removing
> > > >>>>> the "Evolving" annotations on Streams it may be harder to change
> it
> > > >>> again
> > > >>>>> in the future. The extended use cases are that people wanted to
> > have
> > > >>>>> windowed running aggregates on different granularities, e.g.
> "give
> > me
> > > >>> the
> > > >>>>> counts per-minute, per-hour, per-day and per-week", and today in
> > DSL
> > > >> we
> > > >>>>> need to specify that case in multiple aggregate operators, which
> > gets
> > > >> a
> > > >>>>> state store / changelog, etc. And it is possible to optimize it
> as
> > > >> well
> > > >>> to
> > > >>>>> a single state store. Its implementation would be tricky as you
> > need
> > > >> to
> > > >>>>> contain different lengthed windows within your window store but
> > just
> > > >>> from
> > > >>>>> the public API point of view, it could be specified as:
> > > >>>>>
> > > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > > >>>>> "state-store-name");
> > > >>>>>
> > > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > > >>>>>
> > > >>>>> while underlying we are only using a single store
> > "state-store-name"
> > > >> for
> > > >>>>> it.
> > > >>>>>
> > > >>>>>
> > > >>>>> Although this feature is out of the scope of this KIP, I'd like
> to
> > > >>> discuss
> > > >>>>> if we can "leave the door open" to make such changes without
> > > modifying
> > > >>> the
> > > >>>>> public APIs .
> > > >>>>>
> > > >>>>> Guozhang
> > > >>>>>
> > > >>>>>
> > > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > > >>> winkelman.kyle@gmail.com
> > > >>>>>>
> > > >>>>> wrote:
> > > >>>>>
> > > >>>>>> I allow defining a single window/sessionwindow one time when you
> > > >> make
> > > >>>>> the
> > > >>>>>> cogroup call from a KGroupedStream. From then on you are using
> the
> > > >>>>> cogroup
> > > >>>>>> call from with in CogroupedKStream which doesnt accept any
> > > >> additional
> > > >>>>>> windows/sessionwindows.
> > > >>>>>>
> > > >>>>>> Is this what you meant by your question or did I misunderstand?
> > > >>>>>>
> > > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
> > > >> wrote:
> > > >>>>>>
> > > >>>>>> Another question that came to me is on "window alignment": from
> > the
> > > >>> KIP
> > > >>>>> it
> > > >>>>>> seems you are allowing users to specify a (potentially
> different)
> > > >>> window
> > > >>>>>> spec in each co-grouped input stream. So if these window specs
> are
> > > >>>>>> different how should we "align" them with different input
> > streams? I
> > > >>>>> think
> > > >>>>>> it is more natural to only specify on window spec in the
> > > >>>>>>
> > > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> And remove it from the cogroup() functions. WDYT?
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> Guozhang
> > > >>>>>>
> > > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> > wangguoz@gmail.com>
> > > >>>>> wrote:
> > > >>>>>>
> > > >>>>>>> Thanks for the proposal Kyle, this is a quite common use case
> to
> > > >>>>> support
> > > >>>>>>> such multi-way table join (i.e. N source tables with N
> aggregate
> > > >>> func)
> > > >>>>>> with
> > > >>>>>>> a single store and N+1 serdes, I have seen lots of people using
> > > >> the
> > > >>>>>>> low-level PAPI to achieve this goal.
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > > >>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>> wrote:
> > > >>>>>>>
> > > >>>>>>>> I like your point about not handling other cases such as count
> > > >> and
> > > >>>>>> reduce.
> > > >>>>>>>>
> > > >>>>>>>> I think that reduce may not make sense because reduce assumes
> > > >> that
> > > >>>>> the
> > > >>>>>>>> input values are the same as the output values. With cogroup
> > > >> there
> > > >>>>> may
> > > >>>>>> be
> > > >>>>>>>> multiple different input types and then your output type cant
> be
> > > >>>>>> multiple
> > > >>>>>>>> different things. In the case where you have all matching
> value
> > > >>> types
> > > >>>>>> you
> > > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> > > >>>>>>>>
> > > >>>>>>>> As for count I think it is possible to call count on all the
> > > >>>>> individual
> > > >>>>>>>> grouped streams and then do joins. Otherwise we could maybe
> make
> > > >> a
> > > >>>>>> special
> > > >>>>>>>> call in groupedstream for this case. Because in this case we
> > dont
> > > >>>>> need
> > > >>>>>> to
> > > >>>>>>>> do type checking on the values. It could be similar to the
> > > >> current
> > > >>>>> count
> > > >>>>>>>> methods but accept a var args of additonal grouped streams as
> > > >> well
> > > >>>>> and
> > > >>>>>>>> make
> > > >>>>>>>> sure they have a key type of K.
> > > >>>>>>>>
> > > >>>>>>>> The way I have put the kip together is to ensure that we do
> type
> > > >>>>>> checking.
> > > >>>>>>>> I don't see a way we could group them all first and then make
> a
> > > >>> call
> > > >>>>> to
> > > >>>>>>>> count, reduce, or aggregate because with aggregate they would
> > > >> need
> > > >>> to
> > > >>>>>> pass
> > > >>>>>>>> a list of aggregators and we would have no way of type
> checking
> > > >>> that
> > > >>>>>> they
> > > >>>>>>>> match the grouped streams.
> > > >>>>>>>>
> > > >>>>>>>> Thanks,
> > > >>>>>>>> Kyle
> > > >>>>>>>>
> > > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <
> xavier@confluent.io>
> > > >>>>> wrote:
> > > >>>>>>>>
> > > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a very
> > > >>> useful
> > > >>>>>>>>> addition and wanted to provide an additional use-case and
> some
> > > >>> more
> > > >>>>>>>>> comments.
> > > >>>>>>>>>
> > > >>>>>>>>> This is actually a very common analytics use-case in the
> > > >> ad-tech
> > > >>>>>>>> industry.
> > > >>>>>>>>> The typical setup will have an auction stream, an impression
> > > >>>>> stream,
> > > >>>>>>>> and a
> > > >>>>>>>>> click stream. Those three streams need to be combined to
> > > >> compute
> > > >>>>>>>> aggregate
> > > >>>>>>>>> statistics (e.g. impression statistics, and click-through
> > > >> rates),
> > > >>>>>> since
> > > >>>>>>>>> most of the attributes of interest are only present the
> auction
> > > >>>>>> stream.
> > > >>>>>>>>>
> > > >>>>>>>>> A simple way to do this is to co-group all the streams by the
> > > >>>>> auction
> > > >>>>>>>> key,
> > > >>>>>>>>> and process updates to the co-group as events for each stream
> > > >>> come
> > > >>>>> in,
> > > >>>>>>>>> keeping only one value from each stream before sending
> > > >> downstream
> > > >>>>> for
> > > >>>>>>>>> further processing / aggregation.
> > > >>>>>>>>>
> > > >>>>>>>>> One could view the result of that co-group operation as a
> > > >>> "KTable"
> > > >>>>>> with
> > > >>>>>>>>> multiple values per key. The key being the grouping key, and
> > > >> the
> > > >>>>>> values
> > > >>>>>>>>> consisting of one value per stream.
> > > >>>>>>>>>
> > > >>>>>>>>> What I like about Kyle's approach is that allows elegant
> > > >>>>> co-grouping
> > > >>>>>> of
> > > >>>>>>>>> multiple streams without having to worry about the number of
> > > >>>>> streams,
> > > >>>>>>>> and
> > > >>>>>>>>> avoids dealing with Tuple types or other generic interfaces
> > > >> that
> > > >>>>> could
> > > >>>>>>>> get
> > > >>>>>>>>> messy if we wanted to preserve all the value types in the
> > > >>> resulting
> > > >>>>>>>>> co-grouped stream.
> > > >>>>>>>>>
> > > >>>>>>>>> My only concern is that we only allow the cogroup + aggregate
> > > >>>>> combined
> > > >>>>>>>>> operation. This forces the user to build their own tuple
> > > >>>>> serialization
> > > >>>>>>>>> format if they want to preserve the individual input stream
> > > >>> values
> > > >>>>> as
> > > >>>>>> a
> > > >>>>>>>>> group. It also deviates quite a bit from our approach in
> > > >>>>>> KGroupedStream
> > > >>>>>>>>> which offers other operations, such as count and reduce,
> which
> > > >>>>> should
> > > >>>>>>>> also
> > > >>>>>>>>> be applicable to a co-grouped stream.
> > > >>>>>>>>>
> > > >>>>>>>>> Overall I still think this is a really useful addition, but I
> > > >>> feel
> > > >>>>> we
> > > >>>>>>>>> haven't spend much time trying to explore alternative DSLs
> that
> > > >>>>> could
> > > >>>>>>>> maybe
> > > >>>>>>>>> generalize better or match our existing syntax more closely.
> > > >>>>>>>>>
> > > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> > > >>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>>>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>
> > > >>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
> > > >>> streams
> > > >>>>>> realm
> > > >>>>>>>>> that
> > > >>>>>>>>>> I should reach out to for input?
> > > >>>>>>>>>>
> > > >>>>>>>>>> I believe Damian Guy is still planning on reviewing this
> more
> > > >>> in
> > > >>>>>> depth
> > > >>>>>>>>> so I
> > > >>>>>>>>>> will wait for his inputs before continuing.
> > > >>>>>>>>>>
> > > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> > > >> eno.thereska@gmail.com
> > > >>>>
> > > >>>>>>>> wrote:
> > > >>>>>>>>>>
> > > >>>>>>>>>>> Thanks Kyle, good arguments.
> > > >>>>>>>>>>>
> > > >>>>>>>>>>> Eno
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > > >>>>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>>>>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> *- minor: could you add an exact example (similar to what
> > > >>>>> Jay’s
> > > >>>>>>>>> example
> > > >>>>>>>>>>> is,
> > > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> > > >>>>>> concrete?*
> > > >>>>>>>>>>>> I have added a more concrete example to the KIP.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> *- my main concern is that we’re exposing this
> > > >> optimization
> > > >>>>> to
> > > >>>>>> the
> > > >>>>>>>>> DSL.
> > > >>>>>>>>>>> In
> > > >>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> > > >>> and
> > > >>>>> do
> > > >>>>>>>> the
> > > >>>>>>>>>> right
> > > >>>>>>>>>>>> thing under the covers (create just one state store,
> > > >>> arrange
> > > >>>>> the
> > > >>>>>>>>> nodes
> > > >>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> > > >>>>> pieces
> > > >>>>>>>>> (group,
> > > >>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> > > >>>>> like to
> > > >>>>>>>> hear
> > > >>>>>>>>>>> your
> > > >>>>>>>>>>>> thoughts on whether it’s possible to do this optimization
> > > >>>>> with
> > > >>>>>> the
> > > >>>>>>>>>>> current
> > > >>>>>>>>>>>> DSL, at the topology builder level.*
> > > >>>>>>>>>>>> You would have to make a lot of checks to understand if
> > > >> it
> > > >>> is
> > > >>>>>> even
> > > >>>>>>>>>>> possible
> > > >>>>>>>>>>>> to make this optimization:
> > > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> > > >>>>>>>>>>>> 2. None of the intermediate KTables are used for anything
> > > >>>>> else.
> > > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This may be
> > > >>>>>>>> impossible
> > > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> > > >>> topology
> > > >>>>> has
> > > >>>>>>>>>> already
> > > >>>>>>>>>>>> been built.)
> > > >>>>>>>>>>>> You would then need to make decisions during the
> > > >>>>> optimization:
> > > >>>>>>>>>>>> 1. Your new initializer would the composite of all the
> > > >>>>>> individual
> > > >>>>>>>>>>>> initializers and the valueJoiners.
> > > >>>>>>>>>>>> 2. I am having a hard time thinking about how you would
> > > >>> turn
> > > >>>>> the
> > > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> > > >> would
> > > >>>>> work
> > > >>>>>> on
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> final object, but this may be possible.
> > > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> > > >> would
> > > >>>>> be
> > > >>>>>> for
> > > >>>>>>>>> the
> > > >>>>>>>>>>>> aggregate values. None of the declared ones would be
> > > >>>>> guaranteed
> > > >>>>>> to
> > > >>>>>>>>> hold
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>> final object. This would mean you must created a new
> > > >> state
> > > >>>>> store
> > > >>>>>>>> and
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>> created any of the declared ones.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> The main argument I have against it is even if it could
> > > >> be
> > > >>>>> done
> > > >>>>>> I
> > > >>>>>>>>> don't
> > > >>>>>>>>>>>> know that we would want to have this be an optimization
> > > >> in
> > > >>>>> the
> > > >>>>>>>>>> background
> > > >>>>>>>>>>>> because the user would still be required to think about
> > > >> all
> > > >>>>> of
> > > >>>>>> the
> > > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> > > >> about
> > > >>>>> if
> > > >>>>>>>> they
> > > >>>>>>>>>> only
> > > >>>>>>>>>>>> care about the final object.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> > > >>> should
> > > >>>>> be
> > > >>>>>>>> part
> > > >>>>>>>>> of
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> > > >> want
> > > >>> to
> > > >>>>>>>> allow
> > > >>>>>>>>>>> people
> > > >>>>>>>>>>>> to join more than 2 or more streams in an easy way. Right
> > > >>>>> now I
> > > >>>>>>>> don't
> > > >>>>>>>>>>> think
> > > >>>>>>>>>>>> we give them ways of handling this use case easily.
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> *-I think there will be scope for several such
> > > >>> optimizations
> > > >>>>> in
> > > >>>>>>>> the
> > > >>>>>>>>>>> future
> > > >>>>>>>>>>>> and perhaps at some point we need to think about
> > > >> decoupling
> > > >>>>> the
> > > >>>>>>>> 1:1
> > > >>>>>>>>>>> mapping
> > > >>>>>>>>>>>> from the DSL into the physical topology.*
> > > >>>>>>>>>>>> I would argue that cogroup is not just an optimization it
> > > >>> is
> > > >>>>> a
> > > >>>>>> new
> > > >>>>>>>>> way
> > > >>>>>>>>>>> for
> > > >>>>>>>>>>>> the users to look at accomplishing a problem that
> > > >> requires
> > > >>>>>>>> multiple
> > > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> > > >> think
> > > >>>>>> users
> > > >>>>>>>>>> should
> > > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
> > > >>> their
> > > >>>>>>>>>>> initializers,
> > > >>>>>>>>>>>> serdes and stores if all they care about is the final
> > > >>> object.
> > > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> > > >> supply
> > > >>>>>>>> additional
> > > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> > > >> single
> > > >>>>>>>> grouped
> > > >>>>>>>>>>> stream
> > > >>>>>>>>>>>> making an aggregate call. This case is what I view an
> > > >>>>>> optimization
> > > >>>>>>>>> as,
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
> > > >>> just
> > > >>>>> a
> > > >>>>>>>> call
> > > >>>>>>>>> to
> > > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> > > >>>>>>>> KGroupedStream#cogroup.
> > > >>>>>>>>> (I
> > > >>>>>>>>>>>> would prefer to just write a warning saying that this is
> > > >>> not
> > > >>>>> how
> > > >>>>>>>>>> cogroup
> > > >>>>>>>>>>> is
> > > >>>>>>>>>>>> to be used.)
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>> Kyle
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> > > >>>>>>>> eno.thereska@gmail.com
> > > >>>>>>>>>>
> > > >>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>> Hi Kyle,
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> - minor: could you add an exact example (similar to what
> > > >>>>> Jay’s
> > > >>>>>>>>> example
> > > >>>>>>>>>>> is,
> > > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> > > >>>>>> concrete?
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> - my main concern is that we’re exposing this
> > > >> optimization
> > > >>>>> to
> > > >>>>>> the
> > > >>>>>>>>> DSL.
> > > >>>>>>>>>>> In
> > > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> > > >>>>> and do
> > > >>>>>>>> the
> > > >>>>>>>>>>> right
> > > >>>>>>>>>>>>> thing under the covers (create just one state store,
> > > >>> arrange
> > > >>>>>> the
> > > >>>>>>>>> nodes
> > > >>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> > > >>>>> pieces
> > > >>>>>>>>> (group,
> > > >>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> > > >>>>> like
> > > >>>>>> to
> > > >>>>>>>>> hear
> > > >>>>>>>>>>> your
> > > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> > > >> optimization
> > > >>>>> with
> > > >>>>>>>> the
> > > >>>>>>>>>>> current
> > > >>>>>>>>>>>>> DSL, at the topology builder level.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> I think there will be scope for several such
> > > >> optimizations
> > > >>>>> in
> > > >>>>>> the
> > > >>>>>>>>>> future
> > > >>>>>>>>>>>>> and perhaps at some point we need to think about
> > > >>> decoupling
> > > >>>>> the
> > > >>>>>>>> 1:1
> > > >>>>>>>>>>> mapping
> > > >>>>>>>>>>>>> from the DSL into the physical topology.
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>> Thanks
> > > >>>>>>>>>>>>> Eno
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> > > >> jay@confluent.io>
> > > >>>>>> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> > > >>> pretty
> > > >>>>>>>> common.
> > > >>>>>>>>> An
> > > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> > > >> customer
> > > >>>>>>>> profile"
> > > >>>>>>>>> use
> > > >>>>>>>>>>>>> case
> > > >>>>>>>>>>>>>> we often use. In that use case you have a dozen systems
> > > >>>>> each
> > > >>>>>> of
> > > >>>>>>>>> which
> > > >>>>>>>>>>> has
> > > >>>>>>>>>>>>>> some information about your customer (account details,
> > > >>>>>> settings,
> > > >>>>>>>>>>> billing
> > > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> > > >> etc).
> > > >>>>> Your
> > > >>>>>>>> goal
> > > >>>>>>>>> is
> > > >>>>>>>>>>> to
> > > >>>>>>>>>>>>>> join/munge these into a single profile record for each
> > > >>>>>> customer
> > > >>>>>>>>> that
> > > >>>>>>>>>>> has
> > > >>>>>>>>>>>>>> all the relevant info in a usable form and is
> > > >> up-to-date
> > > >>>>> with
> > > >>>>>>>> all
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>> source systems. If you implement that with kstreams as
> > > >> a
> > > >>>>>>>> sequence
> > > >>>>>>>>> of
> > > >>>>>>>>>>>>> joins
> > > >>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
> > > >>>>> tables.
> > > >>>>>>>> But
> > > >>>>>>>>>>> clearly
> > > >>>>>>>>>>>>>> you only need a single stage to group all these things
> > > >>> that
> > > >>>>>> are
> > > >>>>>>>>>> already
> > > >>>>>>>>>>>>>> co-partitioned. A distributed database would do this
> > > >>> under
> > > >>>>> the
> > > >>>>>>>>> covers
> > > >>>>>>>>>>>>> which
> > > >>>>>>>>>>>>>> is arguably better (at least when it does the right
> > > >>> thing)
> > > >>>>> and
> > > >>>>>>>>>> perhaps
> > > >>>>>>>>>>> we
> > > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
> > > >>>>>>>> partitioning
> > > >>>>>>>>> so
> > > >>>>>>>>>> we
> > > >>>>>>>>>>>>> may
> > > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they are
> > > >>>>>> already
> > > >>>>>>>>>>>>>> co-partitioned.
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> -Jay
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> > > >>>>>>>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> > > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> > > >> other
> > > >>>>>>>> platforms
> > > >>>>>>>>>>> like
> > > >>>>>>>>>>>>>>> spark and pig.
> > > >>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> > > >>>>>>>>>>>>> PairRDDFunctions.html
> > > >>>>>>>>>>>>>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
> > > >>>>>>>>>>>>> cogroup_operator.htm
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> > > >>>>> damian.guy@gmail.com>
> > > >>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> Hi Kyle,
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> > > >> outer
> > > >>>>>> join?
> > > >>>>>>>> So
> > > >>>>>>>>> an
> > > >>>>>>>>>>>>> input
> > > >>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
> > > >>> value
> > > >>>>> -
> > > >>>>>> is
> > > >>>>>>>>> that
> > > >>>>>>>>>>>>>>> correct?
> > > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> > > >> current
> > > >>>>>> value,
> > > >>>>>>>>>>>>> aggregates
> > > >>>>>>>>>>>>>>>> and forwards the result.
> > > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit more,
> > > >>>>> but it
> > > >>>>>>>>> seems
> > > >>>>>>>>>>> like
> > > >>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>> could be a useful optimization.
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> > > >>>>>>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> I sure can. I have added the following description
> > > >> to
> > > >>> my
> > > >>>>>>>> KIP. If
> > > >>>>>>>>>>> this
> > > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
> > > >>> time
> > > >>>>> to
> > > >>>>>>>>> build a
> > > >>>>>>>>>>>>>>>> diagram
> > > >>>>>>>>>>>>>>>>> and make more of a step by step description:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Example with Current API:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> > > >>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
> > > >>>>>> initializer1
> > > >>>>>>>> ,
> > > >>>>>>>>>>>>>>>> aggregator1,
> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> > > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> > > >>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
> > > >>>>>> initializer2
> > > >>>>>>>> ,
> > > >>>>>>>>>>>>>>>> aggregator2,
> > > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> > > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> > > >>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
> > > >>>>>> initializer3
> > > >>>>>>>> ,
> > > >>>>>>>>>>>>>>>> aggregator3,
> > > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> > > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> > > >>>>> joinerOneTwoAndThree);
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores, requires
> > > >> 3
> > > >>>>>>>>>> initializers,
> > > >>>>>>>>>>>>>>> and 3
> > > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to user
> > > >> to
> > > >>>>>> define
> > > >>>>>>>>> what
> > > >>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
> > > >> They
> > > >>>>> are
> > > >>>>>>>> left
> > > >>>>>>>>>>> with a
> > > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all the
> > > >>>>> same
> > > >>>>>> as
> > > >>>>>>>> CG
> > > >>>>>>>>>> and
> > > >>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> two
> > > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
> > > >>>>>>>> intermediate
> > > >>>>>>>>>>> states
> > > >>>>>>>>>>>>>>>> such
> > > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> > > >> joiners
> > > >>>>> use
> > > >>>>>>>> those
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>> build
> > > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something the
> > > >>> user
> > > >>>>>>>> could
> > > >>>>>>>>>> avoid
> > > >>>>>>>>>>>>>>>>> thinking about with this KIP.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > > >> will
> > > >>>>> first
> > > >>>>>>>> go
> > > >>>>>>>>>>> through
> > > >>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate from
> > > >>>>>>>> storeName1.
> > > >>>>>>>>>> It
> > > >>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
> > > >>> value
> > > >>>>>> and
> > > >>>>>>>> get
> > > >>>>>>>>>>> sent
> > > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
> > > >> up
> > > >>>>> the
> > > >>>>>>>>> current
> > > >>>>>>>>>>>>> value
> > > >>>>>>>>>>>>>>>> of
> > > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first joiner
> > > >> to
> > > >>>>>>>> calculate
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> second
> > > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> > > >>> additional
> > > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> > > >>> current
> > > >>>>>>>> value of
> > > >>>>>>>>>> the
> > > >>>>>>>>>>>>>>> key
> > > >>>>>>>>>>>>>>>> in
> > > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
> > > >>> final
> > > >>>>>>>>> aggregate
> > > >>>>>>>>>>>>>>> value.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> If you think through all possibilities for incoming
> > > >>>>> topics
> > > >>>>>>>> you
> > > >>>>>>>>>> will
> > > >>>>>>>>>>>>> see
> > > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
> > > >>> three
> > > >>>>>>>> stores
> > > >>>>>>>>>> are
> > > >>>>>>>>>>>>>>>> queried
> > > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates N
> > > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
> > > >> and
> > > >>>>> N-1
> > > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Example with Proposed API:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> > > >>>>> builder.stream("topic1").
> > > >>>>>>>>>>>>>>> groupByKey();
> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> > > >>>>> builder.stream("topic2").
> > > >>>>>>>>>>>>>>> groupByKey();
> > > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> > > >>>>> builder.stream("topic3").
> > > >>>>>>>>>>>>>>> groupByKey();
> > > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> > > >>> grouped1.cogroup(initializer1,
> > > >>>>>>>>>>> aggregator1,
> > > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> > > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> > > >>>>>>>>>>>>>>>>>      .aggregate();
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore, requires 1
> > > >>>>>>>>> initializer,
> > > >>>>>>>>>>> and
> > > >>>>>>>>>>>>> 1
> > > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry about
> > > >>> the
> > > >>>>>>>>>>> intermediate
> > > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think about
> > > >>> is
> > > >>>>> how
> > > >>>>>>>> each
> > > >>>>>>>>>>>>> stream
> > > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > > >> will
> > > >>>>> first
> > > >>>>>>>> go
> > > >>>>>>>>>>> through
> > > >>>>>>>>>>>>>>> a
> > > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate from
> > > >>>>>>>> storeName1.
> > > >>>>>>>>>> It
> > > >>>>>>>>>>>>>>> will
> > > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update the
> > > >>>>> store
> > > >>>>>>>> and
> > > >>>>>>>>>> pass
> > > >>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>> new
> > > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
> > > >>>>>>>> KStreamCogroup
> > > >>>>>>>>>>> which
> > > >>>>>>>>>>>>>>> is
> > > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
> > > >> are
> > > >>>>> done.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> > > >> will
> > > >>>>> only
> > > >>>>>>>> every
> > > >>>>>>>>>>>>>>> create N
> > > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> > > >>>>>>>>>>>>> matthias@confluent.io
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Kyle,
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
> > > >>>>> but I
> > > >>>>>>>> could
> > > >>>>>>>>>> not
> > > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> > > >>> concrete
> > > >>>>>>>> example,
> > > >>>>>>>>>>> like
> > > >>>>>>>>>>>>>>> 3
> > > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected result),
> > > >>> and
> > > >>>>>> show
> > > >>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>> difference between current way to to implement it
> > > >> and
> > > >>>>> the
> > > >>>>>>>>>> proposed
> > > >>>>>>>>>>>>>>> API?
> > > >>>>>>>>>>>>>>>>>> This could also cover the internal processing to
> > > >> see
> > > >>>>> what
> > > >>>>>>>> store
> > > >>>>>>>>>>> calls
> > > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
> > > >> and
> > > >>> it
> > > >>>>>>>> would
> > > >>>>>>>>>> help
> > > >>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>> understand it better.
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> Thanks a lot!
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> -Matthias
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> > > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found here.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> > > >> classes
> > > >>>>> and
> > > >>>>>> get
> > > >>>>>>>>>> around
> > > >>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> > > >> additions.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> > > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> > > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> > > >>>>>>>>> repartitionIfRequired
> > > >>>>>>>>>>>>>>>> method.
> > > >>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
> > > >> one
> > > >>>>>>>> grouped
> > > >>>>>>>>>>> stream
> > > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> > > >> name
> > > >>>>> that
> > > >>>>>>>>> someone
> > > >>>>>>>>>>>>>>> can
> > > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow it
> > > >> to
> > > >>>>> fall
> > > >>>>>>>> back
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped tables?
> > > >>> This
> > > >>>>>>>> would
> > > >>>>>>>>> be
> > > >>>>>>>>>>>>>>>> pretty
> > > >>>>>>>>>>>>>>>>>> easy
> > > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
> > > >>> stores
> > > >>>>>> and
> > > >>>>>>>>>>> windowed
> > > >>>>>>>>>>>>>>>>>> stores
> > > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
> > > >>>>>>>>>>>>>>> KTableWindowAggregate
> > > >>>>>>>>>>>>>>>>>>> implementations.
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>> Kyle
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> > > >>>>>>>>> eno.thereska@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> Eno
> > > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> > > >>>>>>>>> damian.guy@gmail.com>
> > > >>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
> > > >> had
> > > >>>>> the
> > > >>>>>>>>> chance
> > > >>>>>>>>>> to
> > > >>>>>>>>>>>>>>>> look
> > > >>>>>>>>>>>>>>>>>> at
> > > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to look
> > > >>>>> into
> > > >>>>>> it
> > > >>>>>>>>>>>>>>> tomorrow.
> > > >>>>>>>>>>>>>>>>> For
> > > >>>>>>>>>>>>>>>>>>>> the
> > > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against kafka
> > > >>>>> trunk
> > > >>>>>>>> and
> > > >>>>>>>>>> mark
> > > >>>>>>>>>>>>>>> it
> > > >>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>> WIP?
> > > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have done.
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>> Damian
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> > > >> some
> > > >>>>>>>> attention
> > > >>>>>>>>> to
> > > >>>>>>>>>>> my
> > > >>>>>>>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>>>>> as
> > > >>>>>>>>>>>>>>>>>>>> I
> > > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> > > >> This
> > > >>>>> is my
> > > >>>>>>>>> first
> > > >>>>>>>>>>> KIP
> > > >>>>>>>>>>>>>>>> and
> > > >>>>>>>>>>>>>>>>>> my
> > > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so I'm
> > > >>>>> sure I
> > > >>>>>>>> did
> > > >>>>>>>>>>>>>>>> something
> > > >>>>>>>>>>>>>>>>>>>> wrong.
> > > >>>>>>>>>>>>>>>>>>>>>> ;)
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> > > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> > > >>>>>>>>>>>>>>>>>>>> wrote:
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> > > >> discussion
> > > >>>>> about
> > > >>>>>>>>> adding
> > > >>>>>>>>>>>>>>>>> cogroup
> > > >>>>>>>>>>>>>>>>>> to
> > > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> > > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > > >>>>>> confluence/display/KAFKA/KIP-
> > > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
> > > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> > > >>>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>>
> > > >>>>>>>>>>
> > > >>>>>>>>>
> > > >>>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>>
> > > >>>>>>> --
> > > >>>>>>> -- Guozhang
> > > >>>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>>
> > > >>>>>> --
> > > >>>>>> -- Guozhang
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> --
> > > >>>>> -- Guozhang
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Xavier Léauté <xa...@confluent.io>.
I have a minor suggestion to make the API a little bit more symmetric.
I feel it would make more sense to move the initializer and serde to the
final aggregate statement, since the serde only applies to the state store,
and the initializer doesn't bear any relation to the first group in
particular. It would end up looking like this:

KTable<K, CG> cogrouped =
    grouped1.cogroup(aggregator1)
            .cogroup(grouped2, aggregator2)
            .cogroup(grouped3, aggregator3)
            .aggregate(initializer1, aggValueSerde, storeName1);

Alternatively, we could move the the first cogroup() method to
KStreamBuilder, similar to how we have .merge()
and end up with an api that would be even more symmetric.

KStreamBuilder.cogroup(grouped1, aggregator1)
              .cogroup(grouped2, aggregator2)
              .cogroup(grouped3, aggregator3)
              .aggregate(initializer1, aggValueSerde, storeName1);

This doesn't have to be a blocker, but I thought it would make the API just
a tad cleaner.

On Tue, Jun 6, 2017 at 3:59 PM Guozhang Wang <wa...@gmail.com> wrote:

> Kyle,
>
> Thanks a lot for the updated KIP. It looks good to me.
>
>
> Guozhang
>
>
> On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> wrote:
>
> > This makes much more sense to me. +1
> >
> > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <wi...@gmail.com>
> > wrote:
> > >
> > > I have updated the KIP and my PR. Let me know what you think.
> > > To created a cogrouped stream just call cogroup on a KgroupedStream and
> > > supply the initializer, aggValueSerde, and an aggregator. Then continue
> > > adding kgroupedstreams and aggregators. Then call one of the many
> > aggregate
> > > calls to create a KTable.
> > >
> > > Thanks,
> > > Kyle
> > >
> > > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:
> > >
> > >> Hi Kyle,
> > >>
> > >> Thanks for the update. I think just one initializer makes sense as it
> > >> should only be called once per key and generally it is just going to
> > create
> > >> a new instance of whatever the Aggregate class is.
> > >>
> > >> Cheers,
> > >> Damian
> > >>
> > >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <winkelman.kyle@gmail.com
> >
> > >> wrote:
> > >>
> > >>> Hello all,
> > >>>
> > >>> I have spent some more time on this and the best alternative I have
> > come
> > >> up
> > >>> with is:
> > >>> KGroupedStream has a single cogroup call that takes an initializer
> and
> > an
> > >>> aggregator.
> > >>> CogroupedKStream has a cogroup call that takes additional
> groupedStream
> > >>> aggregator pairs.
> > >>> CogroupedKStream has multiple aggregate methods that create the
> > different
> > >>> stores.
> > >>>
> > >>> I plan on updating the kip but I want people's input on if we should
> > have
> > >>> the initializer be passed in once at the beginning or if we should
> > >> instead
> > >>> have the initializer be required for each call to one of the
> aggregate
> > >>> calls. The first makes more sense to me but doesnt allow the user to
> > >>> specify different initializers for different tables.
> > >>>
> > >>> Thanks,
> > >>> Kyle
> > >>>
> > >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <wi...@gmail.com>
> > >>> wrote:
> > >>>
> > >>>> Yea I really like that idea I'll see what I can do to update the kip
> > >> and
> > >>>> my pr when I have some time. I'm not sure how well creating the
> > >>>> kstreamaggregates will go though because at that point I will have
> > >> thrown
> > >>>> away the type of the values. It will be type safe I just may need to
> > >> do a
> > >>>> little forcing.
> > >>>>
> > >>>> Thanks,
> > >>>> Kyle
> > >>>>
> > >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com>
> wrote:
> > >>>>
> > >>>>> Kyle,
> > >>>>>
> > >>>>> Thanks for the explanations, my previous read on the wiki examples
> > was
> > >>>>> wrong.
> > >>>>>
> > >>>>> So I guess my motivation should be "reduced" to: can we move the
> > >> window
> > >>>>> specs param from "KGroupedStream#cogroup(..)" to
> > >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >>>>>
> > >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream from 3
> to
> > >> 2.
> > >>>>> 2. major: this is for extensibility of the APIs, and since we are
> > >>> removing
> > >>>>> the "Evolving" annotations on Streams it may be harder to change it
> > >>> again
> > >>>>> in the future. The extended use cases are that people wanted to
> have
> > >>>>> windowed running aggregates on different granularities, e.g. "give
> me
> > >>> the
> > >>>>> counts per-minute, per-hour, per-day and per-week", and today in
> DSL
> > >> we
> > >>>>> need to specify that case in multiple aggregate operators, which
> gets
> > >> a
> > >>>>> state store / changelog, etc. And it is possible to optimize it as
> > >> well
> > >>> to
> > >>>>> a single state store. Its implementation would be tricky as you
> need
> > >> to
> > >>>>> contain different lengthed windows within your window store but
> just
> > >>> from
> > >>>>> the public API point of view, it could be specified as:
> > >>>>>
> > >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >>>>> "state-store-name");
> > >>>>>
> > >>>>> table1 = stream.aggregate(/*per-minute window*/)
> > >>>>> table2 = stream.aggregate(/*per-hour window*/)
> > >>>>> table3 = stream.aggregate(/*per-day window*/)
> > >>>>>
> > >>>>> while underlying we are only using a single store
> "state-store-name"
> > >> for
> > >>>>> it.
> > >>>>>
> > >>>>>
> > >>>>> Although this feature is out of the scope of this KIP, I'd like to
> > >>> discuss
> > >>>>> if we can "leave the door open" to make such changes without
> > modifying
> > >>> the
> > >>>>> public APIs .
> > >>>>>
> > >>>>> Guozhang
> > >>>>>
> > >>>>>
> > >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > >>> winkelman.kyle@gmail.com
> > >>>>>>
> > >>>>> wrote:
> > >>>>>
> > >>>>>> I allow defining a single window/sessionwindow one time when you
> > >> make
> > >>>>> the
> > >>>>>> cogroup call from a KGroupedStream. From then on you are using the
> > >>>>> cogroup
> > >>>>>> call from with in CogroupedKStream which doesnt accept any
> > >> additional
> > >>>>>> windows/sessionwindows.
> > >>>>>>
> > >>>>>> Is this what you meant by your question or did I misunderstand?
> > >>>>>>
> > >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
> > >> wrote:
> > >>>>>>
> > >>>>>> Another question that came to me is on "window alignment": from
> the
> > >>> KIP
> > >>>>> it
> > >>>>>> seems you are allowing users to specify a (potentially different)
> > >>> window
> > >>>>>> spec in each co-grouped input stream. So if these window specs are
> > >>>>>> different how should we "align" them with different input
> streams? I
> > >>>>> think
> > >>>>>> it is more natural to only specify on window spec in the
> > >>>>>>
> > >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >>>>>>
> > >>>>>>
> > >>>>>> And remove it from the cogroup() functions. WDYT?
> > >>>>>>
> > >>>>>>
> > >>>>>> Guozhang
> > >>>>>>
> > >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <
> wangguoz@gmail.com>
> > >>>>> wrote:
> > >>>>>>
> > >>>>>>> Thanks for the proposal Kyle, this is a quite common use case to
> > >>>>> support
> > >>>>>>> such multi-way table join (i.e. N source tables with N aggregate
> > >>> func)
> > >>>>>> with
> > >>>>>>> a single store and N+1 serdes, I have seen lots of people using
> > >> the
> > >>>>>>> low-level PAPI to achieve this goal.
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >>>>>> winkelman.kyle@gmail.com
> > >>>>>>>> wrote:
> > >>>>>>>
> > >>>>>>>> I like your point about not handling other cases such as count
> > >> and
> > >>>>>> reduce.
> > >>>>>>>>
> > >>>>>>>> I think that reduce may not make sense because reduce assumes
> > >> that
> > >>>>> the
> > >>>>>>>> input values are the same as the output values. With cogroup
> > >> there
> > >>>>> may
> > >>>>>> be
> > >>>>>>>> multiple different input types and then your output type cant be
> > >>>>>> multiple
> > >>>>>>>> different things. In the case where you have all matching value
> > >>> types
> > >>>>>> you
> > >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> > >>>>>>>>
> > >>>>>>>> As for count I think it is possible to call count on all the
> > >>>>> individual
> > >>>>>>>> grouped streams and then do joins. Otherwise we could maybe make
> > >> a
> > >>>>>> special
> > >>>>>>>> call in groupedstream for this case. Because in this case we
> dont
> > >>>>> need
> > >>>>>> to
> > >>>>>>>> do type checking on the values. It could be similar to the
> > >> current
> > >>>>> count
> > >>>>>>>> methods but accept a var args of additonal grouped streams as
> > >> well
> > >>>>> and
> > >>>>>>>> make
> > >>>>>>>> sure they have a key type of K.
> > >>>>>>>>
> > >>>>>>>> The way I have put the kip together is to ensure that we do type
> > >>>>>> checking.
> > >>>>>>>> I don't see a way we could group them all first and then make a
> > >>> call
> > >>>>> to
> > >>>>>>>> count, reduce, or aggregate because with aggregate they would
> > >> need
> > >>> to
> > >>>>>> pass
> > >>>>>>>> a list of aggregators and we would have no way of type checking
> > >>> that
> > >>>>>> they
> > >>>>>>>> match the grouped streams.
> > >>>>>>>>
> > >>>>>>>> Thanks,
> > >>>>>>>> Kyle
> > >>>>>>>>
> > >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <xa...@confluent.io>
> > >>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> Sorry to jump on this thread so late. I agree this is a very
> > >>> useful
> > >>>>>>>>> addition and wanted to provide an additional use-case and some
> > >>> more
> > >>>>>>>>> comments.
> > >>>>>>>>>
> > >>>>>>>>> This is actually a very common analytics use-case in the
> > >> ad-tech
> > >>>>>>>> industry.
> > >>>>>>>>> The typical setup will have an auction stream, an impression
> > >>>>> stream,
> > >>>>>>>> and a
> > >>>>>>>>> click stream. Those three streams need to be combined to
> > >> compute
> > >>>>>>>> aggregate
> > >>>>>>>>> statistics (e.g. impression statistics, and click-through
> > >> rates),
> > >>>>>> since
> > >>>>>>>>> most of the attributes of interest are only present the auction
> > >>>>>> stream.
> > >>>>>>>>>
> > >>>>>>>>> A simple way to do this is to co-group all the streams by the
> > >>>>> auction
> > >>>>>>>> key,
> > >>>>>>>>> and process updates to the co-group as events for each stream
> > >>> come
> > >>>>> in,
> > >>>>>>>>> keeping only one value from each stream before sending
> > >> downstream
> > >>>>> for
> > >>>>>>>>> further processing / aggregation.
> > >>>>>>>>>
> > >>>>>>>>> One could view the result of that co-group operation as a
> > >>> "KTable"
> > >>>>>> with
> > >>>>>>>>> multiple values per key. The key being the grouping key, and
> > >> the
> > >>>>>> values
> > >>>>>>>>> consisting of one value per stream.
> > >>>>>>>>>
> > >>>>>>>>> What I like about Kyle's approach is that allows elegant
> > >>>>> co-grouping
> > >>>>>> of
> > >>>>>>>>> multiple streams without having to worry about the number of
> > >>>>> streams,
> > >>>>>>>> and
> > >>>>>>>>> avoids dealing with Tuple types or other generic interfaces
> > >> that
> > >>>>> could
> > >>>>>>>> get
> > >>>>>>>>> messy if we wanted to preserve all the value types in the
> > >>> resulting
> > >>>>>>>>> co-grouped stream.
> > >>>>>>>>>
> > >>>>>>>>> My only concern is that we only allow the cogroup + aggregate
> > >>>>> combined
> > >>>>>>>>> operation. This forces the user to build their own tuple
> > >>>>> serialization
> > >>>>>>>>> format if they want to preserve the individual input stream
> > >>> values
> > >>>>> as
> > >>>>>> a
> > >>>>>>>>> group. It also deviates quite a bit from our approach in
> > >>>>>> KGroupedStream
> > >>>>>>>>> which offers other operations, such as count and reduce, which
> > >>>>> should
> > >>>>>>>> also
> > >>>>>>>>> be applicable to a co-grouped stream.
> > >>>>>>>>>
> > >>>>>>>>> Overall I still think this is a really useful addition, but I
> > >>> feel
> > >>>>> we
> > >>>>>>>>> haven't spend much time trying to explore alternative DSLs that
> > >>>>> could
> > >>>>>>>> maybe
> > >>>>>>>>> generalize better or match our existing syntax more closely.
> > >>>>>>>>>
> > >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> > >>>>>> winkelman.kyle@gmail.com
> > >>>>>>>>>
> > >>>>>>>>> wrote:
> > >>>>>>>>>
> > >>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
> > >>> streams
> > >>>>>> realm
> > >>>>>>>>> that
> > >>>>>>>>>> I should reach out to for input?
> > >>>>>>>>>>
> > >>>>>>>>>> I believe Damian Guy is still planning on reviewing this more
> > >>> in
> > >>>>>> depth
> > >>>>>>>>> so I
> > >>>>>>>>>> will wait for his inputs before continuing.
> > >>>>>>>>>>
> > >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> > >> eno.thereska@gmail.com
> > >>>>
> > >>>>>>>> wrote:
> > >>>>>>>>>>
> > >>>>>>>>>>> Thanks Kyle, good arguments.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Eno
> > >>>>>>>>>>>
> > >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > >>>>>>>> winkelman.kyle@gmail.com
> > >>>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> *- minor: could you add an exact example (similar to what
> > >>>>> Jay’s
> > >>>>>>>>> example
> > >>>>>>>>>>> is,
> > >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> > >>>>>> concrete?*
> > >>>>>>>>>>>> I have added a more concrete example to the KIP.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> *- my main concern is that we’re exposing this
> > >> optimization
> > >>>>> to
> > >>>>>> the
> > >>>>>>>>> DSL.
> > >>>>>>>>>>> In
> > >>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> > >>> and
> > >>>>> do
> > >>>>>>>> the
> > >>>>>>>>>> right
> > >>>>>>>>>>>> thing under the covers (create just one state store,
> > >>> arrange
> > >>>>> the
> > >>>>>>>>> nodes
> > >>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> > >>>>> pieces
> > >>>>>>>>> (group,
> > >>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> > >>>>> like to
> > >>>>>>>> hear
> > >>>>>>>>>>> your
> > >>>>>>>>>>>> thoughts on whether it’s possible to do this optimization
> > >>>>> with
> > >>>>>> the
> > >>>>>>>>>>> current
> > >>>>>>>>>>>> DSL, at the topology builder level.*
> > >>>>>>>>>>>> You would have to make a lot of checks to understand if
> > >> it
> > >>> is
> > >>>>>> even
> > >>>>>>>>>>> possible
> > >>>>>>>>>>>> to make this optimization:
> > >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> > >>>>>>>>>>>> 2. None of the intermediate KTables are used for anything
> > >>>>> else.
> > >>>>>>>>>>>> 3. None of the intermediate stores are used. (This may be
> > >>>>>>>> impossible
> > >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> > >>> topology
> > >>>>> has
> > >>>>>>>>>> already
> > >>>>>>>>>>>> been built.)
> > >>>>>>>>>>>> You would then need to make decisions during the
> > >>>>> optimization:
> > >>>>>>>>>>>> 1. Your new initializer would the composite of all the
> > >>>>>> individual
> > >>>>>>>>>>>> initializers and the valueJoiners.
> > >>>>>>>>>>>> 2. I am having a hard time thinking about how you would
> > >>> turn
> > >>>>> the
> > >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> > >> would
> > >>>>> work
> > >>>>>> on
> > >>>>>>>>> the
> > >>>>>>>>>>>> final object, but this may be possible.
> > >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> > >> would
> > >>>>> be
> > >>>>>> for
> > >>>>>>>>> the
> > >>>>>>>>>>>> aggregate values. None of the declared ones would be
> > >>>>> guaranteed
> > >>>>>> to
> > >>>>>>>>> hold
> > >>>>>>>>>>> the
> > >>>>>>>>>>>> final object. This would mean you must created a new
> > >> state
> > >>>>> store
> > >>>>>>>> and
> > >>>>>>>>>> not
> > >>>>>>>>>>>> created any of the declared ones.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> The main argument I have against it is even if it could
> > >> be
> > >>>>> done
> > >>>>>> I
> > >>>>>>>>> don't
> > >>>>>>>>>>>> know that we would want to have this be an optimization
> > >> in
> > >>>>> the
> > >>>>>>>>>> background
> > >>>>>>>>>>>> because the user would still be required to think about
> > >> all
> > >>>>> of
> > >>>>>> the
> > >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> > >> about
> > >>>>> if
> > >>>>>>>> they
> > >>>>>>>>>> only
> > >>>>>>>>>>>> care about the final object.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> > >>> should
> > >>>>> be
> > >>>>>>>> part
> > >>>>>>>>> of
> > >>>>>>>>>>> the
> > >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> > >> want
> > >>> to
> > >>>>>>>> allow
> > >>>>>>>>>>> people
> > >>>>>>>>>>>> to join more than 2 or more streams in an easy way. Right
> > >>>>> now I
> > >>>>>>>> don't
> > >>>>>>>>>>> think
> > >>>>>>>>>>>> we give them ways of handling this use case easily.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> *-I think there will be scope for several such
> > >>> optimizations
> > >>>>> in
> > >>>>>>>> the
> > >>>>>>>>>>> future
> > >>>>>>>>>>>> and perhaps at some point we need to think about
> > >> decoupling
> > >>>>> the
> > >>>>>>>> 1:1
> > >>>>>>>>>>> mapping
> > >>>>>>>>>>>> from the DSL into the physical topology.*
> > >>>>>>>>>>>> I would argue that cogroup is not just an optimization it
> > >>> is
> > >>>>> a
> > >>>>>> new
> > >>>>>>>>> way
> > >>>>>>>>>>> for
> > >>>>>>>>>>>> the users to look at accomplishing a problem that
> > >> requires
> > >>>>>>>> multiple
> > >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> > >> think
> > >>>>>> users
> > >>>>>>>>>> should
> > >>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
> > >>> their
> > >>>>>>>>>>> initializers,
> > >>>>>>>>>>>> serdes and stores if all they care about is the final
> > >>> object.
> > >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> > >> supply
> > >>>>>>>> additional
> > >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> > >> single
> > >>>>>>>> grouped
> > >>>>>>>>>>> stream
> > >>>>>>>>>>>> making an aggregate call. This case is what I view an
> > >>>>>> optimization
> > >>>>>>>>> as,
> > >>>>>>>>>> we
> > >>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
> > >>> just
> > >>>>> a
> > >>>>>>>> call
> > >>>>>>>>> to
> > >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> > >>>>>>>> KGroupedStream#cogroup.
> > >>>>>>>>> (I
> > >>>>>>>>>>>> would prefer to just write a warning saying that this is
> > >>> not
> > >>>>> how
> > >>>>>>>>>> cogroup
> > >>>>>>>>>>> is
> > >>>>>>>>>>>> to be used.)
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>> Kyle
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> > >>>>>>>> eno.thereska@gmail.com
> > >>>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>> Hi Kyle,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> - minor: could you add an exact example (similar to what
> > >>>>> Jay’s
> > >>>>>>>>> example
> > >>>>>>>>>>> is,
> > >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> > >>>>>> concrete?
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> - my main concern is that we’re exposing this
> > >> optimization
> > >>>>> to
> > >>>>>> the
> > >>>>>>>>> DSL.
> > >>>>>>>>>>> In
> > >>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> > >>>>> and do
> > >>>>>>>> the
> > >>>>>>>>>>> right
> > >>>>>>>>>>>>> thing under the covers (create just one state store,
> > >>> arrange
> > >>>>>> the
> > >>>>>>>>> nodes
> > >>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> > >>>>> pieces
> > >>>>>>>>> (group,
> > >>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> > >>>>> like
> > >>>>>> to
> > >>>>>>>>> hear
> > >>>>>>>>>>> your
> > >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> > >> optimization
> > >>>>> with
> > >>>>>>>> the
> > >>>>>>>>>>> current
> > >>>>>>>>>>>>> DSL, at the topology builder level.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> I think there will be scope for several such
> > >> optimizations
> > >>>>> in
> > >>>>>> the
> > >>>>>>>>>> future
> > >>>>>>>>>>>>> and perhaps at some point we need to think about
> > >>> decoupling
> > >>>>> the
> > >>>>>>>> 1:1
> > >>>>>>>>>>> mapping
> > >>>>>>>>>>>>> from the DSL into the physical topology.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks
> > >>>>>>>>>>>>> Eno
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> > >> jay@confluent.io>
> > >>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> > >>> pretty
> > >>>>>>>> common.
> > >>>>>>>>> An
> > >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> > >> customer
> > >>>>>>>> profile"
> > >>>>>>>>> use
> > >>>>>>>>>>>>> case
> > >>>>>>>>>>>>>> we often use. In that use case you have a dozen systems
> > >>>>> each
> > >>>>>> of
> > >>>>>>>>> which
> > >>>>>>>>>>> has
> > >>>>>>>>>>>>>> some information about your customer (account details,
> > >>>>>> settings,
> > >>>>>>>>>>> billing
> > >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> > >> etc).
> > >>>>> Your
> > >>>>>>>> goal
> > >>>>>>>>> is
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>> join/munge these into a single profile record for each
> > >>>>>> customer
> > >>>>>>>>> that
> > >>>>>>>>>>> has
> > >>>>>>>>>>>>>> all the relevant info in a usable form and is
> > >> up-to-date
> > >>>>> with
> > >>>>>>>> all
> > >>>>>>>>> the
> > >>>>>>>>>>>>>> source systems. If you implement that with kstreams as
> > >> a
> > >>>>>>>> sequence
> > >>>>>>>>> of
> > >>>>>>>>>>>>> joins
> > >>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
> > >>>>> tables.
> > >>>>>>>> But
> > >>>>>>>>>>> clearly
> > >>>>>>>>>>>>>> you only need a single stage to group all these things
> > >>> that
> > >>>>>> are
> > >>>>>>>>>> already
> > >>>>>>>>>>>>>> co-partitioned. A distributed database would do this
> > >>> under
> > >>>>> the
> > >>>>>>>>> covers
> > >>>>>>>>>>>>> which
> > >>>>>>>>>>>>>> is arguably better (at least when it does the right
> > >>> thing)
> > >>>>> and
> > >>>>>>>>>> perhaps
> > >>>>>>>>>>> we
> > >>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
> > >>>>>>>> partitioning
> > >>>>>>>>> so
> > >>>>>>>>>> we
> > >>>>>>>>>>>>> may
> > >>>>>>>>>>>>>> need an explicit cogroup command that impllies they are
> > >>>>>> already
> > >>>>>>>>>>>>>> co-partitioned.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> -Jay
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> > >>>>>>>>>>> winkelman.kyle@gmail.com
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> > >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> > >> other
> > >>>>>>>> platforms
> > >>>>>>>>>>> like
> > >>>>>>>>>>>>>>> spark and pig.
> > >>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> > >>>>>>>>>>>>> PairRDDFunctions.html
> > >>>>>>>>>>>>>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
> > >>>>>>>>>>>>> cogroup_operator.htm
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> > >>>>> damian.guy@gmail.com>
> > >>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Hi Kyle,
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> > >> outer
> > >>>>>> join?
> > >>>>>>>> So
> > >>>>>>>>> an
> > >>>>>>>>>>>>> input
> > >>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
> > >>> value
> > >>>>> -
> > >>>>>> is
> > >>>>>>>>> that
> > >>>>>>>>>>>>>>> correct?
> > >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> > >> current
> > >>>>>> value,
> > >>>>>>>>>>>>> aggregates
> > >>>>>>>>>>>>>>>> and forwards the result.
> > >>>>>>>>>>>>>>>> I need to look into it and think about it a bit more,
> > >>>>> but it
> > >>>>>>>>> seems
> > >>>>>>>>>>> like
> > >>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>> could be a useful optimization.
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> > >>>>>>>>>> winkelman.kyle@gmail.com
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> I sure can. I have added the following description
> > >> to
> > >>> my
> > >>>>>>>> KIP. If
> > >>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
> > >>> time
> > >>>>> to
> > >>>>>>>>> build a
> > >>>>>>>>>>>>>>>> diagram
> > >>>>>>>>>>>>>>>>> and make more of a step by step description:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Example with Current API:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> > >>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
> > >>>>>> initializer1
> > >>>>>>>> ,
> > >>>>>>>>>>>>>>>> aggregator1,
> > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> > >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> > >>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
> > >>>>>> initializer2
> > >>>>>>>> ,
> > >>>>>>>>>>>>>>>> aggregator2,
> > >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> > >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> > >>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
> > >>>>>> initializer3
> > >>>>>>>> ,
> > >>>>>>>>>>>>>>>> aggregator3,
> > >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> > >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> > >>>>> joinerOneTwoAndThree);
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores, requires
> > >> 3
> > >>>>>>>>>> initializers,
> > >>>>>>>>>>>>>>> and 3
> > >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to user
> > >> to
> > >>>>>> define
> > >>>>>>>>> what
> > >>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
> > >> They
> > >>>>> are
> > >>>>>>>> left
> > >>>>>>>>>>> with a
> > >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all the
> > >>>>> same
> > >>>>>> as
> > >>>>>>>> CG
> > >>>>>>>>>> and
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> two
> > >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
> > >>>>>>>> intermediate
> > >>>>>>>>>>> states
> > >>>>>>>>>>>>>>>> such
> > >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> > >> joiners
> > >>>>> use
> > >>>>>>>> those
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>> build
> > >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something the
> > >>> user
> > >>>>>>>> could
> > >>>>>>>>>> avoid
> > >>>>>>>>>>>>>>>>> thinking about with this KIP.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> will
> > >>>>> first
> > >>>>>>>> go
> > >>>>>>>>>>> through
> > >>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate from
> > >>>>>>>> storeName1.
> > >>>>>>>>>> It
> > >>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
> > >>> value
> > >>>>>> and
> > >>>>>>>> get
> > >>>>>>>>>>> sent
> > >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
> > >> up
> > >>>>> the
> > >>>>>>>>> current
> > >>>>>>>>>>>>> value
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first joiner
> > >> to
> > >>>>>>>> calculate
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>>> second
> > >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> > >>> additional
> > >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> > >>> current
> > >>>>>>>> value of
> > >>>>>>>>>> the
> > >>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
> > >>> final
> > >>>>>>>>> aggregate
> > >>>>>>>>>>>>>>> value.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> If you think through all possibilities for incoming
> > >>>>> topics
> > >>>>>>>> you
> > >>>>>>>>>> will
> > >>>>>>>>>>>>> see
> > >>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
> > >>> three
> > >>>>>>>> stores
> > >>>>>>>>>> are
> > >>>>>>>>>>>>>>>> queried
> > >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates N
> > >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
> > >> and
> > >>>>> N-1
> > >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Example with Proposed API:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> > >>>>> builder.stream("topic1").
> > >>>>>>>>>>>>>>> groupByKey();
> > >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> > >>>>> builder.stream("topic2").
> > >>>>>>>>>>>>>>> groupByKey();
> > >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> > >>>>> builder.stream("topic3").
> > >>>>>>>>>>>>>>> groupByKey();
> > >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> > >>> grouped1.cogroup(initializer1,
> > >>>>>>>>>>> aggregator1,
> > >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> > >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> > >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> > >>>>>>>>>>>>>>>>>      .aggregate();
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore, requires 1
> > >>>>>>>>> initializer,
> > >>>>>>>>>>> and
> > >>>>>>>>>>>>> 1
> > >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry about
> > >>> the
> > >>>>>>>>>>> intermediate
> > >>>>>>>>>>>>>>>>> values and the joiners. All they have to think about
> > >>> is
> > >>>>> how
> > >>>>>>>> each
> > >>>>>>>>>>>>> stream
> > >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> > >> will
> > >>>>> first
> > >>>>>>>> go
> > >>>>>>>>>>> through
> > >>>>>>>>>>>>>>> a
> > >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate from
> > >>>>>>>> storeName1.
> > >>>>>>>>>> It
> > >>>>>>>>>>>>>>> will
> > >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update the
> > >>>>> store
> > >>>>>>>> and
> > >>>>>>>>>> pass
> > >>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
> > >>>>>>>> KStreamCogroup
> > >>>>>>>>>>> which
> > >>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
> > >> are
> > >>>>> done.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> > >> will
> > >>>>> only
> > >>>>>>>> every
> > >>>>>>>>>>>>>>> create N
> > >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> > >>>>>>>>>>>>> matthias@confluent.io
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Kyle,
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
> > >>>>> but I
> > >>>>>>>> could
> > >>>>>>>>>> not
> > >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> > >>> concrete
> > >>>>>>>> example,
> > >>>>>>>>>>> like
> > >>>>>>>>>>>>>>> 3
> > >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected result),
> > >>> and
> > >>>>>> show
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> difference between current way to to implement it
> > >> and
> > >>>>> the
> > >>>>>>>>>> proposed
> > >>>>>>>>>>>>>>> API?
> > >>>>>>>>>>>>>>>>>> This could also cover the internal processing to
> > >> see
> > >>>>> what
> > >>>>>>>> store
> > >>>>>>>>>>> calls
> > >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
> > >> and
> > >>> it
> > >>>>>>>> would
> > >>>>>>>>>> help
> > >>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>> understand it better.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Thanks a lot!
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> > >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found here.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> > >> classes
> > >>>>> and
> > >>>>>> get
> > >>>>>>>>>> around
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> > >> additions.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> > >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> > >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> > >>>>>>>>> repartitionIfRequired
> > >>>>>>>>>>>>>>>> method.
> > >>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
> > >> one
> > >>>>>>>> grouped
> > >>>>>>>>>>> stream
> > >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> > >> name
> > >>>>> that
> > >>>>>>>>> someone
> > >>>>>>>>>>>>>>> can
> > >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow it
> > >> to
> > >>>>> fall
> > >>>>>>>> back
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped tables?
> > >>> This
> > >>>>>>>> would
> > >>>>>>>>> be
> > >>>>>>>>>>>>>>>> pretty
> > >>>>>>>>>>>>>>>>>> easy
> > >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
> > >>> stores
> > >>>>>> and
> > >>>>>>>>>>> windowed
> > >>>>>>>>>>>>>>>>>> stores
> > >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
> > >>>>>>>>>>>>>>> KTableWindowAggregate
> > >>>>>>>>>>>>>>>>>>> implementations.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>> Kyle
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> > >>>>>>>>> eno.thereska@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> Eno
> > >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> > >>>>>>>>> damian.guy@gmail.com>
> > >>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
> > >> had
> > >>>>> the
> > >>>>>>>>> chance
> > >>>>>>>>>> to
> > >>>>>>>>>>>>>>>> look
> > >>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to look
> > >>>>> into
> > >>>>>> it
> > >>>>>>>>>>>>>>> tomorrow.
> > >>>>>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against kafka
> > >>>>> trunk
> > >>>>>>>> and
> > >>>>>>>>>> mark
> > >>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>> WIP?
> > >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have done.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> > >> some
> > >>>>>>>> attention
> > >>>>>>>>> to
> > >>>>>>>>>>> my
> > >>>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> > >> This
> > >>>>> is my
> > >>>>>>>>> first
> > >>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>> my
> > >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so I'm
> > >>>>> sure I
> > >>>>>>>> did
> > >>>>>>>>>>>>>>>> something
> > >>>>>>>>>>>>>>>>>>>> wrong.
> > >>>>>>>>>>>>>>>>>>>>>> ;)
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> > >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> > >> discussion
> > >>>>> about
> > >>>>>>>>> adding
> > >>>>>>>>>>>>>>>>> cogroup
> > >>>>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> > >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > >>>>>> confluence/display/KAFKA/KIP-
> > >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
> > >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> --
> > >>>>>>> -- Guozhang
> > >>>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>>
> > >>>>>> --
> > >>>>>> -- Guozhang
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> --
> > >>>>> -- Guozhang
> > >>>>>
> > >>>>
> > >>>
> > >>
> >
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

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

Thanks a lot for the updated KIP. It looks good to me.


Guozhang


On Fri, Jun 2, 2017 at 5:37 AM, Jim Jagielski <ji...@jagunet.com> wrote:

> This makes much more sense to me. +1
>
> > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <wi...@gmail.com>
> wrote:
> >
> > I have updated the KIP and my PR. Let me know what you think.
> > To created a cogrouped stream just call cogroup on a KgroupedStream and
> > supply the initializer, aggValueSerde, and an aggregator. Then continue
> > adding kgroupedstreams and aggregators. Then call one of the many
> aggregate
> > calls to create a KTable.
> >
> > Thanks,
> > Kyle
> >
> > On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:
> >
> >> Hi Kyle,
> >>
> >> Thanks for the update. I think just one initializer makes sense as it
> >> should only be called once per key and generally it is just going to
> create
> >> a new instance of whatever the Aggregate class is.
> >>
> >> Cheers,
> >> Damian
> >>
> >> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <wi...@gmail.com>
> >> wrote:
> >>
> >>> Hello all,
> >>>
> >>> I have spent some more time on this and the best alternative I have
> come
> >> up
> >>> with is:
> >>> KGroupedStream has a single cogroup call that takes an initializer and
> an
> >>> aggregator.
> >>> CogroupedKStream has a cogroup call that takes additional groupedStream
> >>> aggregator pairs.
> >>> CogroupedKStream has multiple aggregate methods that create the
> different
> >>> stores.
> >>>
> >>> I plan on updating the kip but I want people's input on if we should
> have
> >>> the initializer be passed in once at the beginning or if we should
> >> instead
> >>> have the initializer be required for each call to one of the aggregate
> >>> calls. The first makes more sense to me but doesnt allow the user to
> >>> specify different initializers for different tables.
> >>>
> >>> Thanks,
> >>> Kyle
> >>>
> >>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <wi...@gmail.com>
> >>> wrote:
> >>>
> >>>> Yea I really like that idea I'll see what I can do to update the kip
> >> and
> >>>> my pr when I have some time. I'm not sure how well creating the
> >>>> kstreamaggregates will go though because at that point I will have
> >> thrown
> >>>> away the type of the values. It will be type safe I just may need to
> >> do a
> >>>> little forcing.
> >>>>
> >>>> Thanks,
> >>>> Kyle
> >>>>
> >>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> >>>>
> >>>>> Kyle,
> >>>>>
> >>>>> Thanks for the explanations, my previous read on the wiki examples
> was
> >>>>> wrong.
> >>>>>
> >>>>> So I guess my motivation should be "reduced" to: can we move the
> >> window
> >>>>> specs param from "KGroupedStream#cogroup(..)" to
> >>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
> >>>>>
> >>>>> 1. minor: we can reduce the #.generics in CogroupedKStream from 3 to
> >> 2.
> >>>>> 2. major: this is for extensibility of the APIs, and since we are
> >>> removing
> >>>>> the "Evolving" annotations on Streams it may be harder to change it
> >>> again
> >>>>> in the future. The extended use cases are that people wanted to have
> >>>>> windowed running aggregates on different granularities, e.g. "give me
> >>> the
> >>>>> counts per-minute, per-hour, per-day and per-week", and today in DSL
> >> we
> >>>>> need to specify that case in multiple aggregate operators, which gets
> >> a
> >>>>> state store / changelog, etc. And it is possible to optimize it as
> >> well
> >>> to
> >>>>> a single state store. Its implementation would be tricky as you need
> >> to
> >>>>> contain different lengthed windows within your window store but just
> >>> from
> >>>>> the public API point of view, it could be specified as:
> >>>>>
> >>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
> >>>>> "state-store-name");
> >>>>>
> >>>>> table1 = stream.aggregate(/*per-minute window*/)
> >>>>> table2 = stream.aggregate(/*per-hour window*/)
> >>>>> table3 = stream.aggregate(/*per-day window*/)
> >>>>>
> >>>>> while underlying we are only using a single store "state-store-name"
> >> for
> >>>>> it.
> >>>>>
> >>>>>
> >>>>> Although this feature is out of the scope of this KIP, I'd like to
> >>> discuss
> >>>>> if we can "leave the door open" to make such changes without
> modifying
> >>> the
> >>>>> public APIs .
> >>>>>
> >>>>> Guozhang
> >>>>>
> >>>>>
> >>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> >>> winkelman.kyle@gmail.com
> >>>>>>
> >>>>> wrote:
> >>>>>
> >>>>>> I allow defining a single window/sessionwindow one time when you
> >> make
> >>>>> the
> >>>>>> cogroup call from a KGroupedStream. From then on you are using the
> >>>>> cogroup
> >>>>>> call from with in CogroupedKStream which doesnt accept any
> >> additional
> >>>>>> windows/sessionwindows.
> >>>>>>
> >>>>>> Is this what you meant by your question or did I misunderstand?
> >>>>>>
> >>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
> >> wrote:
> >>>>>>
> >>>>>> Another question that came to me is on "window alignment": from the
> >>> KIP
> >>>>> it
> >>>>>> seems you are allowing users to specify a (potentially different)
> >>> window
> >>>>>> spec in each co-grouped input stream. So if these window specs are
> >>>>>> different how should we "align" them with different input streams? I
> >>>>> think
> >>>>>> it is more natural to only specify on window spec in the
> >>>>>>
> >>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
> >>>>>>
> >>>>>>
> >>>>>> And remove it from the cogroup() functions. WDYT?
> >>>>>>
> >>>>>>
> >>>>>> Guozhang
> >>>>>>
> >>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <wa...@gmail.com>
> >>>>> wrote:
> >>>>>>
> >>>>>>> Thanks for the proposal Kyle, this is a quite common use case to
> >>>>> support
> >>>>>>> such multi-way table join (i.e. N source tables with N aggregate
> >>> func)
> >>>>>> with
> >>>>>>> a single store and N+1 serdes, I have seen lots of people using
> >> the
> >>>>>>> low-level PAPI to achieve this goal.
> >>>>>>>
> >>>>>>>
> >>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> >>>>>> winkelman.kyle@gmail.com
> >>>>>>>> wrote:
> >>>>>>>
> >>>>>>>> I like your point about not handling other cases such as count
> >> and
> >>>>>> reduce.
> >>>>>>>>
> >>>>>>>> I think that reduce may not make sense because reduce assumes
> >> that
> >>>>> the
> >>>>>>>> input values are the same as the output values. With cogroup
> >> there
> >>>>> may
> >>>>>> be
> >>>>>>>> multiple different input types and then your output type cant be
> >>>>>> multiple
> >>>>>>>> different things. In the case where you have all matching value
> >>> types
> >>>>>> you
> >>>>>>>> can do KStreamBuilder#merge followed by the reduce.
> >>>>>>>>
> >>>>>>>> As for count I think it is possible to call count on all the
> >>>>> individual
> >>>>>>>> grouped streams and then do joins. Otherwise we could maybe make
> >> a
> >>>>>> special
> >>>>>>>> call in groupedstream for this case. Because in this case we dont
> >>>>> need
> >>>>>> to
> >>>>>>>> do type checking on the values. It could be similar to the
> >> current
> >>>>> count
> >>>>>>>> methods but accept a var args of additonal grouped streams as
> >> well
> >>>>> and
> >>>>>>>> make
> >>>>>>>> sure they have a key type of K.
> >>>>>>>>
> >>>>>>>> The way I have put the kip together is to ensure that we do type
> >>>>>> checking.
> >>>>>>>> I don't see a way we could group them all first and then make a
> >>> call
> >>>>> to
> >>>>>>>> count, reduce, or aggregate because with aggregate they would
> >> need
> >>> to
> >>>>>> pass
> >>>>>>>> a list of aggregators and we would have no way of type checking
> >>> that
> >>>>>> they
> >>>>>>>> match the grouped streams.
> >>>>>>>>
> >>>>>>>> Thanks,
> >>>>>>>> Kyle
> >>>>>>>>
> >>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <xa...@confluent.io>
> >>>>> wrote:
> >>>>>>>>
> >>>>>>>>> Sorry to jump on this thread so late. I agree this is a very
> >>> useful
> >>>>>>>>> addition and wanted to provide an additional use-case and some
> >>> more
> >>>>>>>>> comments.
> >>>>>>>>>
> >>>>>>>>> This is actually a very common analytics use-case in the
> >> ad-tech
> >>>>>>>> industry.
> >>>>>>>>> The typical setup will have an auction stream, an impression
> >>>>> stream,
> >>>>>>>> and a
> >>>>>>>>> click stream. Those three streams need to be combined to
> >> compute
> >>>>>>>> aggregate
> >>>>>>>>> statistics (e.g. impression statistics, and click-through
> >> rates),
> >>>>>> since
> >>>>>>>>> most of the attributes of interest are only present the auction
> >>>>>> stream.
> >>>>>>>>>
> >>>>>>>>> A simple way to do this is to co-group all the streams by the
> >>>>> auction
> >>>>>>>> key,
> >>>>>>>>> and process updates to the co-group as events for each stream
> >>> come
> >>>>> in,
> >>>>>>>>> keeping only one value from each stream before sending
> >> downstream
> >>>>> for
> >>>>>>>>> further processing / aggregation.
> >>>>>>>>>
> >>>>>>>>> One could view the result of that co-group operation as a
> >>> "KTable"
> >>>>>> with
> >>>>>>>>> multiple values per key. The key being the grouping key, and
> >> the
> >>>>>> values
> >>>>>>>>> consisting of one value per stream.
> >>>>>>>>>
> >>>>>>>>> What I like about Kyle's approach is that allows elegant
> >>>>> co-grouping
> >>>>>> of
> >>>>>>>>> multiple streams without having to worry about the number of
> >>>>> streams,
> >>>>>>>> and
> >>>>>>>>> avoids dealing with Tuple types or other generic interfaces
> >> that
> >>>>> could
> >>>>>>>> get
> >>>>>>>>> messy if we wanted to preserve all the value types in the
> >>> resulting
> >>>>>>>>> co-grouped stream.
> >>>>>>>>>
> >>>>>>>>> My only concern is that we only allow the cogroup + aggregate
> >>>>> combined
> >>>>>>>>> operation. This forces the user to build their own tuple
> >>>>> serialization
> >>>>>>>>> format if they want to preserve the individual input stream
> >>> values
> >>>>> as
> >>>>>> a
> >>>>>>>>> group. It also deviates quite a bit from our approach in
> >>>>>> KGroupedStream
> >>>>>>>>> which offers other operations, such as count and reduce, which
> >>>>> should
> >>>>>>>> also
> >>>>>>>>> be applicable to a co-grouped stream.
> >>>>>>>>>
> >>>>>>>>> Overall I still think this is a really useful addition, but I
> >>> feel
> >>>>> we
> >>>>>>>>> haven't spend much time trying to explore alternative DSLs that
> >>>>> could
> >>>>>>>> maybe
> >>>>>>>>> generalize better or match our existing syntax more closely.
> >>>>>>>>>
> >>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> >>>>>> winkelman.kyle@gmail.com
> >>>>>>>>>
> >>>>>>>>> wrote:
> >>>>>>>>>
> >>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
> >>> streams
> >>>>>> realm
> >>>>>>>>> that
> >>>>>>>>>> I should reach out to for input?
> >>>>>>>>>>
> >>>>>>>>>> I believe Damian Guy is still planning on reviewing this more
> >>> in
> >>>>>> depth
> >>>>>>>>> so I
> >>>>>>>>>> will wait for his inputs before continuing.
> >>>>>>>>>>
> >>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
> >> eno.thereska@gmail.com
> >>>>
> >>>>>>>> wrote:
> >>>>>>>>>>
> >>>>>>>>>>> Thanks Kyle, good arguments.
> >>>>>>>>>>>
> >>>>>>>>>>> Eno
> >>>>>>>>>>>
> >>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> >>>>>>>> winkelman.kyle@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>> *- minor: could you add an exact example (similar to what
> >>>>> Jay’s
> >>>>>>>>> example
> >>>>>>>>>>> is,
> >>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> >>>>>> concrete?*
> >>>>>>>>>>>> I have added a more concrete example to the KIP.
> >>>>>>>>>>>>
> >>>>>>>>>>>> *- my main concern is that we’re exposing this
> >> optimization
> >>>>> to
> >>>>>> the
> >>>>>>>>> DSL.
> >>>>>>>>>>> In
> >>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> >>> and
> >>>>> do
> >>>>>>>> the
> >>>>>>>>>> right
> >>>>>>>>>>>> thing under the covers (create just one state store,
> >>> arrange
> >>>>> the
> >>>>>>>>> nodes
> >>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> >>>>> pieces
> >>>>>>>>> (group,
> >>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> >>>>> like to
> >>>>>>>> hear
> >>>>>>>>>>> your
> >>>>>>>>>>>> thoughts on whether it’s possible to do this optimization
> >>>>> with
> >>>>>> the
> >>>>>>>>>>> current
> >>>>>>>>>>>> DSL, at the topology builder level.*
> >>>>>>>>>>>> You would have to make a lot of checks to understand if
> >> it
> >>> is
> >>>>>> even
> >>>>>>>>>>> possible
> >>>>>>>>>>>> to make this optimization:
> >>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
> >>>>>>>>>>>> 2. None of the intermediate KTables are used for anything
> >>>>> else.
> >>>>>>>>>>>> 3. None of the intermediate stores are used. (This may be
> >>>>>>>> impossible
> >>>>>>>>>>>> especially if they use KafkaStreams#store after the
> >>> topology
> >>>>> has
> >>>>>>>>>> already
> >>>>>>>>>>>> been built.)
> >>>>>>>>>>>> You would then need to make decisions during the
> >>>>> optimization:
> >>>>>>>>>>>> 1. Your new initializer would the composite of all the
> >>>>>> individual
> >>>>>>>>>>>> initializers and the valueJoiners.
> >>>>>>>>>>>> 2. I am having a hard time thinking about how you would
> >>> turn
> >>>>> the
> >>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
> >> would
> >>>>> work
> >>>>>> on
> >>>>>>>>> the
> >>>>>>>>>>>> final object, but this may be possible.
> >>>>>>>>>>>> 3. Which state store would you use? The ones declared
> >> would
> >>>>> be
> >>>>>> for
> >>>>>>>>> the
> >>>>>>>>>>>> aggregate values. None of the declared ones would be
> >>>>> guaranteed
> >>>>>> to
> >>>>>>>>> hold
> >>>>>>>>>>> the
> >>>>>>>>>>>> final object. This would mean you must created a new
> >> state
> >>>>> store
> >>>>>>>> and
> >>>>>>>>>> not
> >>>>>>>>>>>> created any of the declared ones.
> >>>>>>>>>>>>
> >>>>>>>>>>>> The main argument I have against it is even if it could
> >> be
> >>>>> done
> >>>>>> I
> >>>>>>>>> don't
> >>>>>>>>>>>> know that we would want to have this be an optimization
> >> in
> >>>>> the
> >>>>>>>>>> background
> >>>>>>>>>>>> because the user would still be required to think about
> >> all
> >>>>> of
> >>>>>> the
> >>>>>>>>>>>> intermediate values that they shouldn't need to worry
> >> about
> >>>>> if
> >>>>>>>> they
> >>>>>>>>>> only
> >>>>>>>>>>>> care about the final object.
> >>>>>>>>>>>>
> >>>>>>>>>>>> In my opinion cogroup is a common enough case that it
> >>> should
> >>>>> be
> >>>>>>>> part
> >>>>>>>>> of
> >>>>>>>>>>> the
> >>>>>>>>>>>> composable pieces (group, aggregate, join) because we
> >> want
> >>> to
> >>>>>>>> allow
> >>>>>>>>>>> people
> >>>>>>>>>>>> to join more than 2 or more streams in an easy way. Right
> >>>>> now I
> >>>>>>>> don't
> >>>>>>>>>>> think
> >>>>>>>>>>>> we give them ways of handling this use case easily.
> >>>>>>>>>>>>
> >>>>>>>>>>>> *-I think there will be scope for several such
> >>> optimizations
> >>>>> in
> >>>>>>>> the
> >>>>>>>>>>> future
> >>>>>>>>>>>> and perhaps at some point we need to think about
> >> decoupling
> >>>>> the
> >>>>>>>> 1:1
> >>>>>>>>>>> mapping
> >>>>>>>>>>>> from the DSL into the physical topology.*
> >>>>>>>>>>>> I would argue that cogroup is not just an optimization it
> >>> is
> >>>>> a
> >>>>>> new
> >>>>>>>>> way
> >>>>>>>>>>> for
> >>>>>>>>>>>> the users to look at accomplishing a problem that
> >> requires
> >>>>>>>> multiple
> >>>>>>>>>>>> streams. I may sound like a broken record but I don't
> >> think
> >>>>>> users
> >>>>>>>>>> should
> >>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
> >>> their
> >>>>>>>>>>> initializers,
> >>>>>>>>>>>> serdes and stores if all they care about is the final
> >>> object.
> >>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
> >> supply
> >>>>>>>> additional
> >>>>>>>>>>>> streams and aggregators this case is equivalent to a
> >> single
> >>>>>>>> grouped
> >>>>>>>>>>> stream
> >>>>>>>>>>>> making an aggregate call. This case is what I view an
> >>>>>> optimization
> >>>>>>>>> as,
> >>>>>>>>>> we
> >>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
> >>> just
> >>>>> a
> >>>>>>>> call
> >>>>>>>>> to
> >>>>>>>>>>>> KGroupedStream#aggregate instead of calling
> >>>>>>>> KGroupedStream#cogroup.
> >>>>>>>>> (I
> >>>>>>>>>>>> would prefer to just write a warning saying that this is
> >>> not
> >>>>> how
> >>>>>>>>>> cogroup
> >>>>>>>>>>> is
> >>>>>>>>>>>> to be used.)
> >>>>>>>>>>>>
> >>>>>>>>>>>> Thanks,
> >>>>>>>>>>>> Kyle
> >>>>>>>>>>>>
> >>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> >>>>>>>> eno.thereska@gmail.com
> >>>>>>>>>>
> >>>>>>>>>>> wrote:
> >>>>>>>>>>>>
> >>>>>>>>>>>>> Hi Kyle,
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - minor: could you add an exact example (similar to what
> >>>>> Jay’s
> >>>>>>>>> example
> >>>>>>>>>>> is,
> >>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
> >>>>>> concrete?
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> - my main concern is that we’re exposing this
> >> optimization
> >>>>> to
> >>>>>> the
> >>>>>>>>> DSL.
> >>>>>>>>>>> In
> >>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
> >>>>> and do
> >>>>>>>> the
> >>>>>>>>>>> right
> >>>>>>>>>>>>> thing under the covers (create just one state store,
> >>> arrange
> >>>>>> the
> >>>>>>>>> nodes
> >>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
> >>>>> pieces
> >>>>>>>>> (group,
> >>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
> >>>>> like
> >>>>>> to
> >>>>>>>>> hear
> >>>>>>>>>>> your
> >>>>>>>>>>>>> thoughts on whether it’s possible to do this
> >> optimization
> >>>>> with
> >>>>>>>> the
> >>>>>>>>>>> current
> >>>>>>>>>>>>> DSL, at the topology builder level.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> I think there will be scope for several such
> >> optimizations
> >>>>> in
> >>>>>> the
> >>>>>>>>>> future
> >>>>>>>>>>>>> and perhaps at some point we need to think about
> >>> decoupling
> >>>>> the
> >>>>>>>> 1:1
> >>>>>>>>>>> mapping
> >>>>>>>>>>>>> from the DSL into the physical topology.
> >>>>>>>>>>>>>
> >>>>>>>>>>>>> Thanks
> >>>>>>>>>>>>> Eno
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> >> jay@confluent.io>
> >>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> I haven't digested the proposal but the use case is
> >>> pretty
> >>>>>>>> common.
> >>>>>>>>> An
> >>>>>>>>>>>>>> example would be the "customer 360" or "unified
> >> customer
> >>>>>>>> profile"
> >>>>>>>>> use
> >>>>>>>>>>>>> case
> >>>>>>>>>>>>>> we often use. In that use case you have a dozen systems
> >>>>> each
> >>>>>> of
> >>>>>>>>> which
> >>>>>>>>>>> has
> >>>>>>>>>>>>>> some information about your customer (account details,
> >>>>>> settings,
> >>>>>>>>>>> billing
> >>>>>>>>>>>>>> info, customer service contacts, purchase history,
> >> etc).
> >>>>> Your
> >>>>>>>> goal
> >>>>>>>>> is
> >>>>>>>>>>> to
> >>>>>>>>>>>>>> join/munge these into a single profile record for each
> >>>>>> customer
> >>>>>>>>> that
> >>>>>>>>>>> has
> >>>>>>>>>>>>>> all the relevant info in a usable form and is
> >> up-to-date
> >>>>> with
> >>>>>>>> all
> >>>>>>>>> the
> >>>>>>>>>>>>>> source systems. If you implement that with kstreams as
> >> a
> >>>>>>>> sequence
> >>>>>>>>> of
> >>>>>>>>>>>>> joins
> >>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
> >>>>> tables.
> >>>>>>>> But
> >>>>>>>>>>> clearly
> >>>>>>>>>>>>>> you only need a single stage to group all these things
> >>> that
> >>>>>> are
> >>>>>>>>>> already
> >>>>>>>>>>>>>> co-partitioned. A distributed database would do this
> >>> under
> >>>>> the
> >>>>>>>>> covers
> >>>>>>>>>>>>> which
> >>>>>>>>>>>>>> is arguably better (at least when it does the right
> >>> thing)
> >>>>> and
> >>>>>>>>>> perhaps
> >>>>>>>>>>> we
> >>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
> >>>>>>>> partitioning
> >>>>>>>>> so
> >>>>>>>>>> we
> >>>>>>>>>>>>> may
> >>>>>>>>>>>>>> need an explicit cogroup command that impllies they are
> >>>>>> already
> >>>>>>>>>>>>>> co-partitioned.
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> -Jay
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> >>>>>>>>>>> winkelman.kyle@gmail.com
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> Yea thats a good way to look at it.
> >>>>>>>>>>>>>>> I have seen this type of functionality in a couple
> >> other
> >>>>>>>> platforms
> >>>>>>>>>>> like
> >>>>>>>>>>>>>>> spark and pig.
> >>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> >>>>>>>>>>>>> PairRDDFunctions.html
> >>>>>>>>>>>>>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
> >>>>>>>>>>>>> cogroup_operator.htm
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> >>>>> damian.guy@gmail.com>
> >>>>>>>>> wrote:
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> Hi Kyle,
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
> >> outer
> >>>>>> join?
> >>>>>>>> So
> >>>>>>>>> an
> >>>>>>>>>>>>> input
> >>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
> >>> value
> >>>>> -
> >>>>>> is
> >>>>>>>>> that
> >>>>>>>>>>>>>>> correct?
> >>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
> >> current
> >>>>>> value,
> >>>>>>>>>>>>> aggregates
> >>>>>>>>>>>>>>>> and forwards the result.
> >>>>>>>>>>>>>>>> I need to look into it and think about it a bit more,
> >>>>> but it
> >>>>>>>>> seems
> >>>>>>>>>>> like
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> could be a useful optimization.
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> >>>>>>>>>> winkelman.kyle@gmail.com
> >>>>>>>>>>>>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> I sure can. I have added the following description
> >> to
> >>> my
> >>>>>>>> KIP. If
> >>>>>>>>>>> this
> >>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
> >>> time
> >>>>> to
> >>>>>>>>> build a
> >>>>>>>>>>>>>>>> diagram
> >>>>>>>>>>>>>>>>> and make more of a step by step description:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Example with Current API:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
> >>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
> >>>>>> initializer1
> >>>>>>>> ,
> >>>>>>>>>>>>>>>> aggregator1,
> >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
> >>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
> >>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
> >>>>>> initializer2
> >>>>>>>> ,
> >>>>>>>>>>>>>>>> aggregator2,
> >>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
> >>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
> >>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
> >>>>>> initializer3
> >>>>>>>> ,
> >>>>>>>>>>>>>>>> aggregator3,
> >>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
> >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> >>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
> >>>>> joinerOneTwoAndThree);
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores, requires
> >> 3
> >>>>>>>>>> initializers,
> >>>>>>>>>>>>>>> and 3
> >>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to user
> >> to
> >>>>>> define
> >>>>>>>>> what
> >>>>>>>>>>> the
> >>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
> >> They
> >>>>> are
> >>>>>>>> left
> >>>>>>>>>>> with a
> >>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all the
> >>>>> same
> >>>>>> as
> >>>>>>>> CG
> >>>>>>>>>> and
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> two
> >>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
> >>>>>>>> intermediate
> >>>>>>>>>>> states
> >>>>>>>>>>>>>>>> such
> >>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> >> joiners
> >>>>> use
> >>>>>>>> those
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> build
> >>>>>>>>>>>>>>>>> the final aggregate CG value. This is something the
> >>> user
> >>>>>>>> could
> >>>>>>>>>> avoid
> >>>>>>>>>>>>>>>>> thinking about with this KIP.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> >> will
> >>>>> first
> >>>>>>>> go
> >>>>>>>>>>> through
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate from
> >>>>>>>> storeName1.
> >>>>>>>>>> It
> >>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
> >>> value
> >>>>>> and
> >>>>>>>> get
> >>>>>>>>>>> sent
> >>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
> >> up
> >>>>> the
> >>>>>>>>> current
> >>>>>>>>>>>>> value
> >>>>>>>>>>>>>>>> of
> >>>>>>>>>>>>>>>>> the key in storeName2. It will use the first joiner
> >> to
> >>>>>>>> calculate
> >>>>>>>>>> the
> >>>>>>>>>>>>>>>> second
> >>>>>>>>>>>>>>>>> intermediate value, which will go through an
> >>> additional
> >>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
> >>> current
> >>>>>>>> value of
> >>>>>>>>>> the
> >>>>>>>>>>>>>>> key
> >>>>>>>>>>>>>>>> in
> >>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
> >>> final
> >>>>>>>>> aggregate
> >>>>>>>>>>>>>>> value.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> If you think through all possibilities for incoming
> >>>>> topics
> >>>>>>>> you
> >>>>>>>>>> will
> >>>>>>>>>>>>> see
> >>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
> >>> three
> >>>>>>>> stores
> >>>>>>>>>> are
> >>>>>>>>>>>>>>>> queried
> >>>>>>>>>>>>>>>>> and all of the joiners must get used.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates N
> >>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
> >> and
> >>>>> N-1
> >>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Example with Proposed API:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
> >>>>> builder.stream("topic1").
> >>>>>>>>>>>>>>> groupByKey();
> >>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
> >>>>> builder.stream("topic2").
> >>>>>>>>>>>>>>> groupByKey();
> >>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
> >>>>> builder.stream("topic3").
> >>>>>>>>>>>>>>> groupByKey();
> >>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
> >>> grouped1.cogroup(initializer1,
> >>>>>>>>>>> aggregator1,
> >>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
> >>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
> >>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
> >>>>>>>>>>>>>>>>>      .aggregate();
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore, requires 1
> >>>>>>>>> initializer,
> >>>>>>>>>>> and
> >>>>>>>>>>>>> 1
> >>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry about
> >>> the
> >>>>>>>>>>> intermediate
> >>>>>>>>>>>>>>>>> values and the joiners. All they have to think about
> >>> is
> >>>>> how
> >>>>>>>> each
> >>>>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
> >> will
> >>>>> first
> >>>>>>>> go
> >>>>>>>>>>> through
> >>>>>>>>>>>>>>> a
> >>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate from
> >>>>>>>> storeName1.
> >>>>>>>>>> It
> >>>>>>>>>>>>>>> will
> >>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update the
> >>>>> store
> >>>>>>>> and
> >>>>>>>>>> pass
> >>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>> new
> >>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
> >>>>>>>> KStreamCogroup
> >>>>>>>>>>> which
> >>>>>>>>>>>>>>> is
> >>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
> >> are
> >>>>> done.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
> >> will
> >>>>> only
> >>>>>>>> every
> >>>>>>>>>>>>>>> create N
> >>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> >>>>>>>>>>>>> matthias@confluent.io
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Kyle,
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
> >>>>> but I
> >>>>>>>> could
> >>>>>>>>>> not
> >>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
> >>> concrete
> >>>>>>>> example,
> >>>>>>>>>>> like
> >>>>>>>>>>>>>>> 3
> >>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected result),
> >>> and
> >>>>>> show
> >>>>>>>>> the
> >>>>>>>>>>>>>>>>>> difference between current way to to implement it
> >> and
> >>>>> the
> >>>>>>>>>> proposed
> >>>>>>>>>>>>>>> API?
> >>>>>>>>>>>>>>>>>> This could also cover the internal processing to
> >> see
> >>>>> what
> >>>>>>>> store
> >>>>>>>>>>> calls
> >>>>>>>>>>>>>>>>>> would be required for both approaches etc.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
> >> and
> >>> it
> >>>>>>>> would
> >>>>>>>>>> help
> >>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>> understand it better.
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> Thanks a lot!
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> -Matthias
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> >>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found here.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
> >> classes
> >>>>> and
> >>>>>> get
> >>>>>>>>>> around
> >>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>> writing documentation for the public api
> >> additions.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
> >>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
> >>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
> >>>>>>>>> repartitionIfRequired
> >>>>>>>>>>>>>>>> method.
> >>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
> >> one
> >>>>>>>> grouped
> >>>>>>>>>>> stream
> >>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
> >> name
> >>>>> that
> >>>>>>>>> someone
> >>>>>>>>>>>>>>> can
> >>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow it
> >> to
> >>>>> fall
> >>>>>>>> back
> >>>>>>>>>> to
> >>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>> KGroupedStream.name?
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped tables?
> >>> This
> >>>>>>>> would
> >>>>>>>>> be
> >>>>>>>>>>>>>>>> pretty
> >>>>>>>>>>>>>>>>>> easy
> >>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
> >>> stores
> >>>>>> and
> >>>>>>>>>>> windowed
> >>>>>>>>>>>>>>>>>> stores
> >>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
> >>>>>>>>>>>>>>> KTableWindowAggregate
> >>>>>>>>>>>>>>>>>>> implementations.
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>> Kyle
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> >>>>>>>>> eno.thereska@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> Eno
> >>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> >>>>>>>>> damian.guy@gmail.com>
> >>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Hi Kyle,
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
> >> had
> >>>>> the
> >>>>>>>>> chance
> >>>>>>>>>> to
> >>>>>>>>>>>>>>>> look
> >>>>>>>>>>>>>>>>>> at
> >>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to look
> >>>>> into
> >>>>>> it
> >>>>>>>>>>>>>>> tomorrow.
> >>>>>>>>>>>>>>>>> For
> >>>>>>>>>>>>>>>>>>>> the
> >>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against kafka
> >>>>> trunk
> >>>>>>>> and
> >>>>>>>>>> mark
> >>>>>>>>>>>>>>> it
> >>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> WIP?
> >>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have done.
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>> Damian
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
> >> some
> >>>>>>>> attention
> >>>>>>>>> to
> >>>>>>>>>>> my
> >>>>>>>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>>>> as
> >>>>>>>>>>>>>>>>>>>> I
> >>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
> >> This
> >>>>> is my
> >>>>>>>>> first
> >>>>>>>>>>> KIP
> >>>>>>>>>>>>>>>> and
> >>>>>>>>>>>>>>>>>> my
> >>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so I'm
> >>>>> sure I
> >>>>>>>> did
> >>>>>>>>>>>>>>>> something
> >>>>>>>>>>>>>>>>>>>> wrong.
> >>>>>>>>>>>>>>>>>>>>>> ;)
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> >>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
> >>>>>>>>>>>>>>>>>>>> wrote:
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Hello all,
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
> >> discussion
> >>>>> about
> >>>>>>>>> adding
> >>>>>>>>>>>>>>>>> cogroup
> >>>>>>>>>>>>>>>>>> to
> >>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
> >>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> >>>>>> confluence/display/KAFKA/KIP-
> >>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
> >>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>> Thanks,
> >>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
> >>>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>>
> >>>>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> --
> >>>>>>> -- Guozhang
> >>>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> --
> >>>>>> -- Guozhang
> >>>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> -- Guozhang
> >>>>>
> >>>>
> >>>
> >>
>
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Jim Jagielski <ji...@jaguNET.com>.
This makes much more sense to me. +1

> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <wi...@gmail.com> wrote:
> 
> I have updated the KIP and my PR. Let me know what you think.
> To created a cogrouped stream just call cogroup on a KgroupedStream and
> supply the initializer, aggValueSerde, and an aggregator. Then continue
> adding kgroupedstreams and aggregators. Then call one of the many aggregate
> calls to create a KTable.
> 
> Thanks,
> Kyle
> 
> On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:
> 
>> Hi Kyle,
>> 
>> Thanks for the update. I think just one initializer makes sense as it
>> should only be called once per key and generally it is just going to create
>> a new instance of whatever the Aggregate class is.
>> 
>> Cheers,
>> Damian
>> 
>> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <wi...@gmail.com>
>> wrote:
>> 
>>> Hello all,
>>> 
>>> I have spent some more time on this and the best alternative I have come
>> up
>>> with is:
>>> KGroupedStream has a single cogroup call that takes an initializer and an
>>> aggregator.
>>> CogroupedKStream has a cogroup call that takes additional groupedStream
>>> aggregator pairs.
>>> CogroupedKStream has multiple aggregate methods that create the different
>>> stores.
>>> 
>>> I plan on updating the kip but I want people's input on if we should have
>>> the initializer be passed in once at the beginning or if we should
>> instead
>>> have the initializer be required for each call to one of the aggregate
>>> calls. The first makes more sense to me but doesnt allow the user to
>>> specify different initializers for different tables.
>>> 
>>> Thanks,
>>> Kyle
>>> 
>>> On May 24, 2017 7:46 PM, "Kyle Winkelman" <wi...@gmail.com>
>>> wrote:
>>> 
>>>> Yea I really like that idea I'll see what I can do to update the kip
>> and
>>>> my pr when I have some time. I'm not sure how well creating the
>>>> kstreamaggregates will go though because at that point I will have
>> thrown
>>>> away the type of the values. It will be type safe I just may need to
>> do a
>>>> little forcing.
>>>> 
>>>> Thanks,
>>>> Kyle
>>>> 
>>>> On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
>>>> 
>>>>> Kyle,
>>>>> 
>>>>> Thanks for the explanations, my previous read on the wiki examples was
>>>>> wrong.
>>>>> 
>>>>> So I guess my motivation should be "reduced" to: can we move the
>> window
>>>>> specs param from "KGroupedStream#cogroup(..)" to
>>>>> "CogroupedKStream#aggregate(..)", and my motivations are:
>>>>> 
>>>>> 1. minor: we can reduce the #.generics in CogroupedKStream from 3 to
>> 2.
>>>>> 2. major: this is for extensibility of the APIs, and since we are
>>> removing
>>>>> the "Evolving" annotations on Streams it may be harder to change it
>>> again
>>>>> in the future. The extended use cases are that people wanted to have
>>>>> windowed running aggregates on different granularities, e.g. "give me
>>> the
>>>>> counts per-minute, per-hour, per-day and per-week", and today in DSL
>> we
>>>>> need to specify that case in multiple aggregate operators, which gets
>> a
>>>>> state store / changelog, etc. And it is possible to optimize it as
>> well
>>> to
>>>>> a single state store. Its implementation would be tricky as you need
>> to
>>>>> contain different lengthed windows within your window store but just
>>> from
>>>>> the public API point of view, it could be specified as:
>>>>> 
>>>>> CogroupedKStream stream = stream1.cogroup(stream2, ...
>>>>> "state-store-name");
>>>>> 
>>>>> table1 = stream.aggregate(/*per-minute window*/)
>>>>> table2 = stream.aggregate(/*per-hour window*/)
>>>>> table3 = stream.aggregate(/*per-day window*/)
>>>>> 
>>>>> while underlying we are only using a single store "state-store-name"
>> for
>>>>> it.
>>>>> 
>>>>> 
>>>>> Although this feature is out of the scope of this KIP, I'd like to
>>> discuss
>>>>> if we can "leave the door open" to make such changes without modifying
>>> the
>>>>> public APIs .
>>>>> 
>>>>> Guozhang
>>>>> 
>>>>> 
>>>>> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
>>> winkelman.kyle@gmail.com
>>>>>> 
>>>>> wrote:
>>>>> 
>>>>>> I allow defining a single window/sessionwindow one time when you
>> make
>>>>> the
>>>>>> cogroup call from a KGroupedStream. From then on you are using the
>>>>> cogroup
>>>>>> call from with in CogroupedKStream which doesnt accept any
>> additional
>>>>>> windows/sessionwindows.
>>>>>> 
>>>>>> Is this what you meant by your question or did I misunderstand?
>>>>>> 
>>>>>> On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
>> wrote:
>>>>>> 
>>>>>> Another question that came to me is on "window alignment": from the
>>> KIP
>>>>> it
>>>>>> seems you are allowing users to specify a (potentially different)
>>> window
>>>>>> spec in each co-grouped input stream. So if these window specs are
>>>>>> different how should we "align" them with different input streams? I
>>>>> think
>>>>>> it is more natural to only specify on window spec in the
>>>>>> 
>>>>>> KTable<RK, V> CogroupedKStream#aggregate(Windows);
>>>>>> 
>>>>>> 
>>>>>> And remove it from the cogroup() functions. WDYT?
>>>>>> 
>>>>>> 
>>>>>> Guozhang
>>>>>> 
>>>>>> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <wa...@gmail.com>
>>>>> wrote:
>>>>>> 
>>>>>>> Thanks for the proposal Kyle, this is a quite common use case to
>>>>> support
>>>>>>> such multi-way table join (i.e. N source tables with N aggregate
>>> func)
>>>>>> with
>>>>>>> a single store and N+1 serdes, I have seen lots of people using
>> the
>>>>>>> low-level PAPI to achieve this goal.
>>>>>>> 
>>>>>>> 
>>>>>>> On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com
>>>>>>>> wrote:
>>>>>>> 
>>>>>>>> I like your point about not handling other cases such as count
>> and
>>>>>> reduce.
>>>>>>>> 
>>>>>>>> I think that reduce may not make sense because reduce assumes
>> that
>>>>> the
>>>>>>>> input values are the same as the output values. With cogroup
>> there
>>>>> may
>>>>>> be
>>>>>>>> multiple different input types and then your output type cant be
>>>>>> multiple
>>>>>>>> different things. In the case where you have all matching value
>>> types
>>>>>> you
>>>>>>>> can do KStreamBuilder#merge followed by the reduce.
>>>>>>>> 
>>>>>>>> As for count I think it is possible to call count on all the
>>>>> individual
>>>>>>>> grouped streams and then do joins. Otherwise we could maybe make
>> a
>>>>>> special
>>>>>>>> call in groupedstream for this case. Because in this case we dont
>>>>> need
>>>>>> to
>>>>>>>> do type checking on the values. It could be similar to the
>> current
>>>>> count
>>>>>>>> methods but accept a var args of additonal grouped streams as
>> well
>>>>> and
>>>>>>>> make
>>>>>>>> sure they have a key type of K.
>>>>>>>> 
>>>>>>>> The way I have put the kip together is to ensure that we do type
>>>>>> checking.
>>>>>>>> I don't see a way we could group them all first and then make a
>>> call
>>>>> to
>>>>>>>> count, reduce, or aggregate because with aggregate they would
>> need
>>> to
>>>>>> pass
>>>>>>>> a list of aggregators and we would have no way of type checking
>>> that
>>>>>> they
>>>>>>>> match the grouped streams.
>>>>>>>> 
>>>>>>>> Thanks,
>>>>>>>> Kyle
>>>>>>>> 
>>>>>>>> On May 19, 2017 11:42 AM, "Xavier Léauté" <xa...@confluent.io>
>>>>> wrote:
>>>>>>>> 
>>>>>>>>> Sorry to jump on this thread so late. I agree this is a very
>>> useful
>>>>>>>>> addition and wanted to provide an additional use-case and some
>>> more
>>>>>>>>> comments.
>>>>>>>>> 
>>>>>>>>> This is actually a very common analytics use-case in the
>> ad-tech
>>>>>>>> industry.
>>>>>>>>> The typical setup will have an auction stream, an impression
>>>>> stream,
>>>>>>>> and a
>>>>>>>>> click stream. Those three streams need to be combined to
>> compute
>>>>>>>> aggregate
>>>>>>>>> statistics (e.g. impression statistics, and click-through
>> rates),
>>>>>> since
>>>>>>>>> most of the attributes of interest are only present the auction
>>>>>> stream.
>>>>>>>>> 
>>>>>>>>> A simple way to do this is to co-group all the streams by the
>>>>> auction
>>>>>>>> key,
>>>>>>>>> and process updates to the co-group as events for each stream
>>> come
>>>>> in,
>>>>>>>>> keeping only one value from each stream before sending
>> downstream
>>>>> for
>>>>>>>>> further processing / aggregation.
>>>>>>>>> 
>>>>>>>>> One could view the result of that co-group operation as a
>>> "KTable"
>>>>>> with
>>>>>>>>> multiple values per key. The key being the grouping key, and
>> the
>>>>>> values
>>>>>>>>> consisting of one value per stream.
>>>>>>>>> 
>>>>>>>>> What I like about Kyle's approach is that allows elegant
>>>>> co-grouping
>>>>>> of
>>>>>>>>> multiple streams without having to worry about the number of
>>>>> streams,
>>>>>>>> and
>>>>>>>>> avoids dealing with Tuple types or other generic interfaces
>> that
>>>>> could
>>>>>>>> get
>>>>>>>>> messy if we wanted to preserve all the value types in the
>>> resulting
>>>>>>>>> co-grouped stream.
>>>>>>>>> 
>>>>>>>>> My only concern is that we only allow the cogroup + aggregate
>>>>> combined
>>>>>>>>> operation. This forces the user to build their own tuple
>>>>> serialization
>>>>>>>>> format if they want to preserve the individual input stream
>>> values
>>>>> as
>>>>>> a
>>>>>>>>> group. It also deviates quite a bit from our approach in
>>>>>> KGroupedStream
>>>>>>>>> which offers other operations, such as count and reduce, which
>>>>> should
>>>>>>>> also
>>>>>>>>> be applicable to a co-grouped stream.
>>>>>>>>> 
>>>>>>>>> Overall I still think this is a really useful addition, but I
>>> feel
>>>>> we
>>>>>>>>> haven't spend much time trying to explore alternative DSLs that
>>>>> could
>>>>>>>> maybe
>>>>>>>>> generalize better or match our existing syntax more closely.
>>>>>>>>> 
>>>>>>>>> On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
>>>>>> winkelman.kyle@gmail.com
>>>>>>>>> 
>>>>>>>>> wrote:
>>>>>>>>> 
>>>>>>>>>> Eno, is there anyone else that is an expert in the kafka
>>> streams
>>>>>> realm
>>>>>>>>> that
>>>>>>>>>> I should reach out to for input?
>>>>>>>>>> 
>>>>>>>>>> I believe Damian Guy is still planning on reviewing this more
>>> in
>>>>>> depth
>>>>>>>>> so I
>>>>>>>>>> will wait for his inputs before continuing.
>>>>>>>>>> 
>>>>>>>>>> On May 9, 2017 7:30 AM, "Eno Thereska" <
>> eno.thereska@gmail.com
>>>> 
>>>>>>>> wrote:
>>>>>>>>>> 
>>>>>>>>>>> Thanks Kyle, good arguments.
>>>>>>>>>>> 
>>>>>>>>>>> Eno
>>>>>>>>>>> 
>>>>>>>>>>>> On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>> 
>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> *- minor: could you add an exact example (similar to what
>>>>> Jay’s
>>>>>>>>> example
>>>>>>>>>>> is,
>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
>>>>>> concrete?*
>>>>>>>>>>>> I have added a more concrete example to the KIP.
>>>>>>>>>>>> 
>>>>>>>>>>>> *- my main concern is that we’re exposing this
>> optimization
>>>>> to
>>>>>> the
>>>>>>>>> DSL.
>>>>>>>>>>> In
>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
>>> and
>>>>> do
>>>>>>>> the
>>>>>>>>>> right
>>>>>>>>>>>> thing under the covers (create just one state store,
>>> arrange
>>>>> the
>>>>>>>>> nodes
>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
>>>>> pieces
>>>>>>>>> (group,
>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
>>>>> like to
>>>>>>>> hear
>>>>>>>>>>> your
>>>>>>>>>>>> thoughts on whether it’s possible to do this optimization
>>>>> with
>>>>>> the
>>>>>>>>>>> current
>>>>>>>>>>>> DSL, at the topology builder level.*
>>>>>>>>>>>> You would have to make a lot of checks to understand if
>> it
>>> is
>>>>>> even
>>>>>>>>>>> possible
>>>>>>>>>>>> to make this optimization:
>>>>>>>>>>>> 1. Make sure they are all KTableKTableOuterJoins
>>>>>>>>>>>> 2. None of the intermediate KTables are used for anything
>>>>> else.
>>>>>>>>>>>> 3. None of the intermediate stores are used. (This may be
>>>>>>>> impossible
>>>>>>>>>>>> especially if they use KafkaStreams#store after the
>>> topology
>>>>> has
>>>>>>>>>> already
>>>>>>>>>>>> been built.)
>>>>>>>>>>>> You would then need to make decisions during the
>>>>> optimization:
>>>>>>>>>>>> 1. Your new initializer would the composite of all the
>>>>>> individual
>>>>>>>>>>>> initializers and the valueJoiners.
>>>>>>>>>>>> 2. I am having a hard time thinking about how you would
>>> turn
>>>>> the
>>>>>>>>>>>> aggregators and valueJoiners into an aggregator that
>> would
>>>>> work
>>>>>> on
>>>>>>>>> the
>>>>>>>>>>>> final object, but this may be possible.
>>>>>>>>>>>> 3. Which state store would you use? The ones declared
>> would
>>>>> be
>>>>>> for
>>>>>>>>> the
>>>>>>>>>>>> aggregate values. None of the declared ones would be
>>>>> guaranteed
>>>>>> to
>>>>>>>>> hold
>>>>>>>>>>> the
>>>>>>>>>>>> final object. This would mean you must created a new
>> state
>>>>> store
>>>>>>>> and
>>>>>>>>>> not
>>>>>>>>>>>> created any of the declared ones.
>>>>>>>>>>>> 
>>>>>>>>>>>> The main argument I have against it is even if it could
>> be
>>>>> done
>>>>>> I
>>>>>>>>> don't
>>>>>>>>>>>> know that we would want to have this be an optimization
>> in
>>>>> the
>>>>>>>>>> background
>>>>>>>>>>>> because the user would still be required to think about
>> all
>>>>> of
>>>>>> the
>>>>>>>>>>>> intermediate values that they shouldn't need to worry
>> about
>>>>> if
>>>>>>>> they
>>>>>>>>>> only
>>>>>>>>>>>> care about the final object.
>>>>>>>>>>>> 
>>>>>>>>>>>> In my opinion cogroup is a common enough case that it
>>> should
>>>>> be
>>>>>>>> part
>>>>>>>>> of
>>>>>>>>>>> the
>>>>>>>>>>>> composable pieces (group, aggregate, join) because we
>> want
>>> to
>>>>>>>> allow
>>>>>>>>>>> people
>>>>>>>>>>>> to join more than 2 or more streams in an easy way. Right
>>>>> now I
>>>>>>>> don't
>>>>>>>>>>> think
>>>>>>>>>>>> we give them ways of handling this use case easily.
>>>>>>>>>>>> 
>>>>>>>>>>>> *-I think there will be scope for several such
>>> optimizations
>>>>> in
>>>>>>>> the
>>>>>>>>>>> future
>>>>>>>>>>>> and perhaps at some point we need to think about
>> decoupling
>>>>> the
>>>>>>>> 1:1
>>>>>>>>>>> mapping
>>>>>>>>>>>> from the DSL into the physical topology.*
>>>>>>>>>>>> I would argue that cogroup is not just an optimization it
>>> is
>>>>> a
>>>>>> new
>>>>>>>>> way
>>>>>>>>>>> for
>>>>>>>>>>>> the users to look at accomplishing a problem that
>> requires
>>>>>>>> multiple
>>>>>>>>>>>> streams. I may sound like a broken record but I don't
>> think
>>>>>> users
>>>>>>>>>> should
>>>>>>>>>>>> have to build the N-1 intermediate tables and deal with
>>> their
>>>>>>>>>>> initializers,
>>>>>>>>>>>> serdes and stores if all they care about is the final
>>> object.
>>>>>>>>>>>> Now if for example someone uses cogroup but doesn't
>> supply
>>>>>>>> additional
>>>>>>>>>>>> streams and aggregators this case is equivalent to a
>> single
>>>>>>>> grouped
>>>>>>>>>>> stream
>>>>>>>>>>>> making an aggregate call. This case is what I view an
>>>>>> optimization
>>>>>>>>> as,
>>>>>>>>>> we
>>>>>>>>>>>> could remove the KStreamCogroup and act as if there was
>>> just
>>>>> a
>>>>>>>> call
>>>>>>>>> to
>>>>>>>>>>>> KGroupedStream#aggregate instead of calling
>>>>>>>> KGroupedStream#cogroup.
>>>>>>>>> (I
>>>>>>>>>>>> would prefer to just write a warning saying that this is
>>> not
>>>>> how
>>>>>>>>>> cogroup
>>>>>>>>>>> is
>>>>>>>>>>>> to be used.)
>>>>>>>>>>>> 
>>>>>>>>>>>> Thanks,
>>>>>>>>>>>> Kyle
>>>>>>>>>>>> 
>>>>>>>>>>>> On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
>>>>>>>> eno.thereska@gmail.com
>>>>>>>>>> 
>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for the KIP again. A couple of comments:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> - minor: could you add an exact example (similar to what
>>>>> Jay’s
>>>>>>>>> example
>>>>>>>>>>> is,
>>>>>>>>>>>>> or like your Spark/Pig pointers had) to make this super
>>>>>> concrete?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> - my main concern is that we’re exposing this
>> optimization
>>>>> to
>>>>>> the
>>>>>>>>> DSL.
>>>>>>>>>>> In
>>>>>>>>>>>>> an ideal world, an optimizer would take the existing DSL
>>>>> and do
>>>>>>>> the
>>>>>>>>>>> right
>>>>>>>>>>>>> thing under the covers (create just one state store,
>>> arrange
>>>>>> the
>>>>>>>>> nodes
>>>>>>>>>>>>> etc). The original DSL had a bunch of small, composable
>>>>> pieces
>>>>>>>>> (group,
>>>>>>>>>>>>> aggregate, join) that this proposal groups together. I’d
>>>>> like
>>>>>> to
>>>>>>>>> hear
>>>>>>>>>>> your
>>>>>>>>>>>>> thoughts on whether it’s possible to do this
>> optimization
>>>>> with
>>>>>>>> the
>>>>>>>>>>> current
>>>>>>>>>>>>> DSL, at the topology builder level.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> I think there will be scope for several such
>> optimizations
>>>>> in
>>>>>> the
>>>>>>>>>> future
>>>>>>>>>>>>> and perhaps at some point we need to think about
>>> decoupling
>>>>> the
>>>>>>>> 1:1
>>>>>>>>>>> mapping
>>>>>>>>>>>>> from the DSL into the physical topology.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks
>>>>>>>>>>>>> Eno
>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On May 5, 2017, at 4:39 PM, Jay Kreps <
>> jay@confluent.io>
>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> I haven't digested the proposal but the use case is
>>> pretty
>>>>>>>> common.
>>>>>>>>> An
>>>>>>>>>>>>>> example would be the "customer 360" or "unified
>> customer
>>>>>>>> profile"
>>>>>>>>> use
>>>>>>>>>>>>> case
>>>>>>>>>>>>>> we often use. In that use case you have a dozen systems
>>>>> each
>>>>>> of
>>>>>>>>> which
>>>>>>>>>>> has
>>>>>>>>>>>>>> some information about your customer (account details,
>>>>>> settings,
>>>>>>>>>>> billing
>>>>>>>>>>>>>> info, customer service contacts, purchase history,
>> etc).
>>>>> Your
>>>>>>>> goal
>>>>>>>>> is
>>>>>>>>>>> to
>>>>>>>>>>>>>> join/munge these into a single profile record for each
>>>>>> customer
>>>>>>>>> that
>>>>>>>>>>> has
>>>>>>>>>>>>>> all the relevant info in a usable form and is
>> up-to-date
>>>>> with
>>>>>>>> all
>>>>>>>>> the
>>>>>>>>>>>>>> source systems. If you implement that with kstreams as
>> a
>>>>>>>> sequence
>>>>>>>>> of
>>>>>>>>>>>>> joins
>>>>>>>>>>>>>> i think today we'd fully materialize N-1 intermediate
>>>>> tables.
>>>>>>>> But
>>>>>>>>>>> clearly
>>>>>>>>>>>>>> you only need a single stage to group all these things
>>> that
>>>>>> are
>>>>>>>>>> already
>>>>>>>>>>>>>> co-partitioned. A distributed database would do this
>>> under
>>>>> the
>>>>>>>>> covers
>>>>>>>>>>>>> which
>>>>>>>>>>>>>> is arguably better (at least when it does the right
>>> thing)
>>>>> and
>>>>>>>>>> perhaps
>>>>>>>>>>> we
>>>>>>>>>>>>>> could do the same thing but I'm not sure we know the
>>>>>>>> partitioning
>>>>>>>>> so
>>>>>>>>>> we
>>>>>>>>>>>>> may
>>>>>>>>>>>>>> need an explicit cogroup command that impllies they are
>>>>>> already
>>>>>>>>>>>>>> co-partitioned.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> -Jay
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> Yea thats a good way to look at it.
>>>>>>>>>>>>>>> I have seen this type of functionality in a couple
>> other
>>>>>>>> platforms
>>>>>>>>>>> like
>>>>>>>>>>>>>>> spark and pig.
>>>>>>>>>>>>>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
>>>>>>>>>>>>> PairRDDFunctions.html
>>>>>>>>>>>>>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
>>>>>>>>>>>>> cogroup_operator.htm
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On May 5, 2017 7:43 AM, "Damian Guy" <
>>>>> damian.guy@gmail.com>
>>>>>>>>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> If i'm reading this correctly it is like an N way
>> outer
>>>>>> join?
>>>>>>>> So
>>>>>>>>> an
>>>>>>>>>>>>> input
>>>>>>>>>>>>>>>> on any stream will always produce a new aggregated
>>> value
>>>>> -
>>>>>> is
>>>>>>>>> that
>>>>>>>>>>>>>>> correct?
>>>>>>>>>>>>>>>> Effectively, each Aggregator just looks up the
>> current
>>>>>> value,
>>>>>>>>>>>>> aggregates
>>>>>>>>>>>>>>>> and forwards the result.
>>>>>>>>>>>>>>>> I need to look into it and think about it a bit more,
>>>>> but it
>>>>>>>>> seems
>>>>>>>>>>> like
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> could be a useful optimization.
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> I sure can. I have added the following description
>> to
>>> my
>>>>>>>> KIP. If
>>>>>>>>>>> this
>>>>>>>>>>>>>>>>> doesn't help let me know and I will take some more
>>> time
>>>>> to
>>>>>>>>> build a
>>>>>>>>>>>>>>>> diagram
>>>>>>>>>>>>>>>>> and make more of a step by step description:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Example with Current API:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> KTable<K, V1> table1 =
>>>>>>>>>>>>>>>>> builder.stream("topic1").groupByKey().aggregate(
>>>>>> initializer1
>>>>>>>> ,
>>>>>>>>>>>>>>>> aggregator1,
>>>>>>>>>>>>>>>>> aggValueSerde1, storeName1);
>>>>>>>>>>>>>>>>> KTable<K, V2> table2 =
>>>>>>>>>>>>>>>>> builder.stream("topic2").groupByKey().aggregate(
>>>>>> initializer2
>>>>>>>> ,
>>>>>>>>>>>>>>>> aggregator2,
>>>>>>>>>>>>>>>>> aggValueSerde2, storeName2);
>>>>>>>>>>>>>>>>> KTable<K, V3> table3 =
>>>>>>>>>>>>>>>>> builder.stream("topic3").groupByKey().aggregate(
>>>>>> initializer3
>>>>>>>> ,
>>>>>>>>>>>>>>>> aggregator3,
>>>>>>>>>>>>>>>>> aggValueSerde3, storeName3);
>>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
>>>>>>>>>>>>>>>>> joinerOneAndTwo).outerJoin(table3,
>>>>> joinerOneTwoAndThree);
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> As you can see this creates 3 StateStores, requires
>> 3
>>>>>>>>>> initializers,
>>>>>>>>>>>>>>> and 3
>>>>>>>>>>>>>>>>> aggValueSerdes. This also adds the pressure to user
>> to
>>>>>> define
>>>>>>>>> what
>>>>>>>>>>> the
>>>>>>>>>>>>>>>>> intermediate values are going to be (V1, V2, V3).
>> They
>>>>> are
>>>>>>>> left
>>>>>>>>>>> with a
>>>>>>>>>>>>>>>>> couple choices, first to make V1, V2, and V3 all the
>>>>> same
>>>>>> as
>>>>>>>> CG
>>>>>>>>>> and
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> two
>>>>>>>>>>>>>>>>> joiners are more like mergers, or second make them
>>>>>>>> intermediate
>>>>>>>>>>> states
>>>>>>>>>>>>>>>> such
>>>>>>>>>>>>>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
>> joiners
>>>>> use
>>>>>>>> those
>>>>>>>>>> to
>>>>>>>>>>>>>>> build
>>>>>>>>>>>>>>>>> the final aggregate CG value. This is something the
>>> user
>>>>>>>> could
>>>>>>>>>> avoid
>>>>>>>>>>>>>>>>> thinking about with this KIP.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>> will
>>>>> first
>>>>>>>> go
>>>>>>>>>>> through
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> KStreamAggregate grabbing the current aggregate from
>>>>>>>> storeName1.
>>>>>>>>>> It
>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> produce this in the form of the first intermediate
>>> value
>>>>>> and
>>>>>>>> get
>>>>>>>>>>> sent
>>>>>>>>>>>>>>>>> through a KTableKTableOuterJoin where it will look
>> up
>>>>> the
>>>>>>>>> current
>>>>>>>>>>>>> value
>>>>>>>>>>>>>>>> of
>>>>>>>>>>>>>>>>> the key in storeName2. It will use the first joiner
>> to
>>>>>>>> calculate
>>>>>>>>>> the
>>>>>>>>>>>>>>>> second
>>>>>>>>>>>>>>>>> intermediate value, which will go through an
>>> additional
>>>>>>>>>>>>>>>>> KTableKTableOuterJoin. Here it will look up the
>>> current
>>>>>>>> value of
>>>>>>>>>> the
>>>>>>>>>>>>>>> key
>>>>>>>>>>>>>>>> in
>>>>>>>>>>>>>>>>> storeName3 and use the second joiner to build the
>>> final
>>>>>>>>> aggregate
>>>>>>>>>>>>>>> value.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> If you think through all possibilities for incoming
>>>>> topics
>>>>>>>> you
>>>>>>>>>> will
>>>>>>>>>>>>> see
>>>>>>>>>>>>>>>>> that no matter which topic it comes in through all
>>> three
>>>>>>>> stores
>>>>>>>>>> are
>>>>>>>>>>>>>>>> queried
>>>>>>>>>>>>>>>>> and all of the joiners must get used.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Topology wise for N incoming streams this creates N
>>>>>>>>>>>>>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
>> and
>>>>> N-1
>>>>>>>>>>>>>>>>> KTableKTableJoinMergers.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Example with Proposed API:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> KGroupedStream<K, V1> grouped1 =
>>>>> builder.stream("topic1").
>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>> KGroupedStream<K, V2> grouped2 =
>>>>> builder.stream("topic2").
>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>> KGroupedStream<K, V3> grouped3 =
>>>>> builder.stream("topic3").
>>>>>>>>>>>>>>> groupByKey();
>>>>>>>>>>>>>>>>> KTable<K, CG> cogrouped =
>>> grouped1.cogroup(initializer1,
>>>>>>>>>>> aggregator1,
>>>>>>>>>>>>>>>>> aggValueSerde1, storeName1)
>>>>>>>>>>>>>>>>>      .cogroup(grouped2, aggregator2)
>>>>>>>>>>>>>>>>>      .cogroup(grouped3, aggregator3)
>>>>>>>>>>>>>>>>>      .aggregate();
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> As you can see this creates 1 StateStore, requires 1
>>>>>>>>> initializer,
>>>>>>>>>>> and
>>>>>>>>>>>>> 1
>>>>>>>>>>>>>>>>> aggValueSerde. The user no longer has to worry about
>>> the
>>>>>>>>>>> intermediate
>>>>>>>>>>>>>>>>> values and the joiners. All they have to think about
>>> is
>>>>> how
>>>>>>>> each
>>>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>> impacts the creation of the final CG object.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> When a new input arrives lets say at "topic1" it
>> will
>>>>> first
>>>>>>>> go
>>>>>>>>>>> through
>>>>>>>>>>>>>>> a
>>>>>>>>>>>>>>>>> KStreamAggreagte and grab the current aggregate from
>>>>>>>> storeName1.
>>>>>>>>>> It
>>>>>>>>>>>>>>> will
>>>>>>>>>>>>>>>>> add its incoming object to the aggregate, update the
>>>>> store
>>>>>>>> and
>>>>>>>>>> pass
>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>> new
>>>>>>>>>>>>>>>>> aggregate on. This new aggregate goes through the
>>>>>>>> KStreamCogroup
>>>>>>>>>>> which
>>>>>>>>>>>>>>> is
>>>>>>>>>>>>>>>>> pretty much just a pass through processor and you
>> are
>>>>> done.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> Topology wise for N incoming streams the new api
>> will
>>>>> only
>>>>>>>> every
>>>>>>>>>>>>>>> create N
>>>>>>>>>>>>>>>>> KStreamAggregates and 1 KStreamCogroup.
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
>>>>>>>>>>>>> matthias@confluent.io
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Kyle,
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
>>>>> but I
>>>>>>>> could
>>>>>>>>>> not
>>>>>>>>>>>>>>>>>> follow completely. Could you maybe add a more
>>> concrete
>>>>>>>> example,
>>>>>>>>>>> like
>>>>>>>>>>>>>>> 3
>>>>>>>>>>>>>>>>>> streams with 3 records each (plus expected result),
>>> and
>>>>>> show
>>>>>>>>> the
>>>>>>>>>>>>>>>>>> difference between current way to to implement it
>> and
>>>>> the
>>>>>>>>>> proposed
>>>>>>>>>>>>>>> API?
>>>>>>>>>>>>>>>>>> This could also cover the internal processing to
>> see
>>>>> what
>>>>>>>> store
>>>>>>>>>>> calls
>>>>>>>>>>>>>>>>>> would be required for both approaches etc.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> I think, it's pretty advanced stuff you propose,
>> and
>>> it
>>>>>>>> would
>>>>>>>>>> help
>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>> understand it better.
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> Thanks a lot!
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> -Matthias
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
>>>>>>>>>>>>>>>>>>> I have made a pull request. It can be found here.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> https://github.com/apache/kafka/pull/2975
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> I plan to write some more unit tests for my
>> classes
>>>>> and
>>>>>> get
>>>>>>>>>> around
>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>> writing documentation for the public api
>> additions.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> One thing I was curious about is during the
>>>>>>>>>>>>>>>>>> KCogroupedStreamImpl#aggregate
>>>>>>>>>>>>>>>>>>> method I pass null to the KGroupedStream#
>>>>>>>>> repartitionIfRequired
>>>>>>>>>>>>>>>> method.
>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>> can't supply the store name because if more than
>> one
>>>>>>>> grouped
>>>>>>>>>>> stream
>>>>>>>>>>>>>>>>>>> repartitions an error is thrown. Is there some
>> name
>>>>> that
>>>>>>>>> someone
>>>>>>>>>>>>>>> can
>>>>>>>>>>>>>>>>>>> recommend or should I leave the null and allow it
>> to
>>>>> fall
>>>>>>>> back
>>>>>>>>>> to
>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>> KGroupedStream.name?
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Should this be expanded to handle grouped tables?
>>> This
>>>>>>>> would
>>>>>>>>> be
>>>>>>>>>>>>>>>> pretty
>>>>>>>>>>>>>>>>>> easy
>>>>>>>>>>>>>>>>>>> for a normal aggregate but one allowing session
>>> stores
>>>>>> and
>>>>>>>>>>> windowed
>>>>>>>>>>>>>>>>>> stores
>>>>>>>>>>>>>>>>>>> would required KTableSessionWindowAggregate and
>>>>>>>>>>>>>>> KTableWindowAggregate
>>>>>>>>>>>>>>>>>>> implementations.
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>> Kyle
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
>>>>>>>>> eno.thereska@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> I’ll look as well asap, sorry, been swamped.
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> Eno
>>>>>>>>>>>>>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
>>>>>>>>> damian.guy@gmail.com>
>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Hi Kyle,
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP. I apologize that i haven't
>> had
>>>>> the
>>>>>>>>> chance
>>>>>>>>>> to
>>>>>>>>>>>>>>>> look
>>>>>>>>>>>>>>>>>> at
>>>>>>>>>>>>>>>>>>>>> the KIP yet, but will schedule some time to look
>>>>> into
>>>>>> it
>>>>>>>>>>>>>>> tomorrow.
>>>>>>>>>>>>>>>>> For
>>>>>>>>>>>>>>>>>>>> the
>>>>>>>>>>>>>>>>>>>>> implementation, can you raise a PR against kafka
>>>>> trunk
>>>>>>>> and
>>>>>>>>>> mark
>>>>>>>>>>>>>>> it
>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> WIP?
>>>>>>>>>>>>>>>>>>>>> It will be easier to review what you have done.
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>> Damian
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> I am replying to this in hopes it will draw
>> some
>>>>>>>> attention
>>>>>>>>> to
>>>>>>>>>>> my
>>>>>>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>>>> as
>>>>>>>>>>>>>>>>>>>> I
>>>>>>>>>>>>>>>>>>>>>> haven't heard from anyone in a couple days.
>> This
>>>>> is my
>>>>>>>>> first
>>>>>>>>>>> KIP
>>>>>>>>>>>>>>>> and
>>>>>>>>>>>>>>>>>> my
>>>>>>>>>>>>>>>>>>>>>> first large contribution to the project so I'm
>>>>> sure I
>>>>>>>> did
>>>>>>>>>>>>>>>> something
>>>>>>>>>>>>>>>>>>>> wrong.
>>>>>>>>>>>>>>>>>>>>>> ;)
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
>>>>>>>>>>>>>>>> winkelman.kyle@gmail.com>
>>>>>>>>>>>>>>>>>>>> wrote:
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Hello all,
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> I have created KIP-150 to facilitate
>> discussion
>>>>> about
>>>>>>>>> adding
>>>>>>>>>>>>>>>>> cogroup
>>>>>>>>>>>>>>>>>> to
>>>>>>>>>>>>>>>>>>>>>>> the streams DSL.
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Please find the KIP here:
>>>>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
>>>>>> confluence/display/KAFKA/KIP-
>>>>>>>>>>>>>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Please find my initial implementation here:
>>>>>>>>>>>>>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>>> Thanks,
>>>>>>>>>>>>>>>>>>>>>>> Kyle Winkelman
>>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>>> --
>>>>>>> -- Guozhang
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> -- Guozhang
>>>>>> 
>>>>> 
>>>>> 
>>>>> 
>>>>> --
>>>>> -- Guozhang
>>>>> 
>>>> 
>>> 
>> 


Re: [DISCUSS] KIP-150 - Kafka-Streams Cogroup

Posted by Kyle Winkelman <wi...@gmail.com>.
I have updated the KIP and my PR. Let me know what you think.
To created a cogrouped stream just call cogroup on a KgroupedStream and
supply the initializer, aggValueSerde, and an aggregator. Then continue
adding kgroupedstreams and aggregators. Then call one of the many aggregate
calls to create a KTable.

Thanks,
Kyle

On Jun 1, 2017 4:03 AM, "Damian Guy" <da...@gmail.com> wrote:

> Hi Kyle,
>
> Thanks for the update. I think just one initializer makes sense as it
> should only be called once per key and generally it is just going to create
> a new instance of whatever the Aggregate class is.
>
> Cheers,
> Damian
>
> On Wed, 31 May 2017 at 20:09 Kyle Winkelman <wi...@gmail.com>
> wrote:
>
> > Hello all,
> >
> > I have spent some more time on this and the best alternative I have come
> up
> > with is:
> > KGroupedStream has a single cogroup call that takes an initializer and an
> > aggregator.
> > CogroupedKStream has a cogroup call that takes additional groupedStream
> > aggregator pairs.
> > CogroupedKStream has multiple aggregate methods that create the different
> > stores.
> >
> > I plan on updating the kip but I want people's input on if we should have
> > the initializer be passed in once at the beginning or if we should
> instead
> > have the initializer be required for each call to one of the aggregate
> > calls. The first makes more sense to me but doesnt allow the user to
> > specify different initializers for different tables.
> >
> > Thanks,
> > Kyle
> >
> > On May 24, 2017 7:46 PM, "Kyle Winkelman" <wi...@gmail.com>
> > wrote:
> >
> > > Yea I really like that idea I'll see what I can do to update the kip
> and
> > > my pr when I have some time. I'm not sure how well creating the
> > > kstreamaggregates will go though because at that point I will have
> thrown
> > > away the type of the values. It will be type safe I just may need to
> do a
> > > little forcing.
> > >
> > > Thanks,
> > > Kyle
> > >
> > > On May 24, 2017 3:28 PM, "Guozhang Wang" <wa...@gmail.com> wrote:
> > >
> > >> Kyle,
> > >>
> > >> Thanks for the explanations, my previous read on the wiki examples was
> > >> wrong.
> > >>
> > >> So I guess my motivation should be "reduced" to: can we move the
> window
> > >> specs param from "KGroupedStream#cogroup(..)" to
> > >> "CogroupedKStream#aggregate(..)", and my motivations are:
> > >>
> > >> 1. minor: we can reduce the #.generics in CogroupedKStream from 3 to
> 2.
> > >> 2. major: this is for extensibility of the APIs, and since we are
> > removing
> > >> the "Evolving" annotations on Streams it may be harder to change it
> > again
> > >> in the future. The extended use cases are that people wanted to have
> > >> windowed running aggregates on different granularities, e.g. "give me
> > the
> > >> counts per-minute, per-hour, per-day and per-week", and today in DSL
> we
> > >> need to specify that case in multiple aggregate operators, which gets
> a
> > >> state store / changelog, etc. And it is possible to optimize it as
> well
> > to
> > >> a single state store. Its implementation would be tricky as you need
> to
> > >> contain different lengthed windows within your window store but just
> > from
> > >> the public API point of view, it could be specified as:
> > >>
> > >> CogroupedKStream stream = stream1.cogroup(stream2, ...
> > >> "state-store-name");
> > >>
> > >> table1 = stream.aggregate(/*per-minute window*/)
> > >> table2 = stream.aggregate(/*per-hour window*/)
> > >> table3 = stream.aggregate(/*per-day window*/)
> > >>
> > >> while underlying we are only using a single store "state-store-name"
> for
> > >> it.
> > >>
> > >>
> > >> Although this feature is out of the scope of this KIP, I'd like to
> > discuss
> > >> if we can "leave the door open" to make such changes without modifying
> > the
> > >> public APIs .
> > >>
> > >> Guozhang
> > >>
> > >>
> > >> On Wed, May 24, 2017 at 3:57 AM, Kyle Winkelman <
> > winkelman.kyle@gmail.com
> > >> >
> > >> wrote:
> > >>
> > >> > I allow defining a single window/sessionwindow one time when you
> make
> > >> the
> > >> > cogroup call from a KGroupedStream. From then on you are using the
> > >> cogroup
> > >> > call from with in CogroupedKStream which doesnt accept any
> additional
> > >> > windows/sessionwindows.
> > >> >
> > >> > Is this what you meant by your question or did I misunderstand?
> > >> >
> > >> > On May 23, 2017 9:33 PM, "Guozhang Wang" <wa...@gmail.com>
> wrote:
> > >> >
> > >> > Another question that came to me is on "window alignment": from the
> > KIP
> > >> it
> > >> > seems you are allowing users to specify a (potentially different)
> > window
> > >> > spec in each co-grouped input stream. So if these window specs are
> > >> > different how should we "align" them with different input streams? I
> > >> think
> > >> > it is more natural to only specify on window spec in the
> > >> >
> > >> > KTable<RK, V> CogroupedKStream#aggregate(Windows);
> > >> >
> > >> >
> > >> > And remove it from the cogroup() functions. WDYT?
> > >> >
> > >> >
> > >> > Guozhang
> > >> >
> > >> > On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >> >
> > >> > > Thanks for the proposal Kyle, this is a quite common use case to
> > >> support
> > >> > > such multi-way table join (i.e. N source tables with N aggregate
> > func)
> > >> > with
> > >> > > a single store and N+1 serdes, I have seen lots of people using
> the
> > >> > > low-level PAPI to achieve this goal.
> > >> > >
> > >> > >
> > >> > > On Fri, May 19, 2017 at 10:04 AM, Kyle Winkelman <
> > >> > winkelman.kyle@gmail.com
> > >> > > > wrote:
> > >> > >
> > >> > >> I like your point about not handling other cases such as count
> and
> > >> > reduce.
> > >> > >>
> > >> > >> I think that reduce may not make sense because reduce assumes
> that
> > >> the
> > >> > >> input values are the same as the output values. With cogroup
> there
> > >> may
> > >> > be
> > >> > >> multiple different input types and then your output type cant be
> > >> > multiple
> > >> > >> different things. In the case where you have all matching value
> > types
> > >> > you
> > >> > >> can do KStreamBuilder#merge followed by the reduce.
> > >> > >>
> > >> > >> As for count I think it is possible to call count on all the
> > >> individual
> > >> > >> grouped streams and then do joins. Otherwise we could maybe make
> a
> > >> > special
> > >> > >> call in groupedstream for this case. Because in this case we dont
> > >> need
> > >> > to
> > >> > >> do type checking on the values. It could be similar to the
> current
> > >> count
> > >> > >> methods but accept a var args of additonal grouped streams as
> well
> > >> and
> > >> > >> make
> > >> > >> sure they have a key type of K.
> > >> > >>
> > >> > >> The way I have put the kip together is to ensure that we do type
> > >> > checking.
> > >> > >> I don't see a way we could group them all first and then make a
> > call
> > >> to
> > >> > >> count, reduce, or aggregate because with aggregate they would
> need
> > to
> > >> > pass
> > >> > >> a list of aggregators and we would have no way of type checking
> > that
> > >> > they
> > >> > >> match the grouped streams.
> > >> > >>
> > >> > >> Thanks,
> > >> > >> Kyle
> > >> > >>
> > >> > >> On May 19, 2017 11:42 AM, "Xavier Léauté" <xa...@confluent.io>
> > >> wrote:
> > >> > >>
> > >> > >> > Sorry to jump on this thread so late. I agree this is a very
> > useful
> > >> > >> > addition and wanted to provide an additional use-case and some
> > more
> > >> > >> > comments.
> > >> > >> >
> > >> > >> > This is actually a very common analytics use-case in the
> ad-tech
> > >> > >> industry.
> > >> > >> > The typical setup will have an auction stream, an impression
> > >> stream,
> > >> > >> and a
> > >> > >> > click stream. Those three streams need to be combined to
> compute
> > >> > >> aggregate
> > >> > >> > statistics (e.g. impression statistics, and click-through
> rates),
> > >> > since
> > >> > >> > most of the attributes of interest are only present the auction
> > >> > stream.
> > >> > >> >
> > >> > >> > A simple way to do this is to co-group all the streams by the
> > >> auction
> > >> > >> key,
> > >> > >> > and process updates to the co-group as events for each stream
> > come
> > >> in,
> > >> > >> > keeping only one value from each stream before sending
> downstream
> > >> for
> > >> > >> > further processing / aggregation.
> > >> > >> >
> > >> > >> > One could view the result of that co-group operation as a
> > "KTable"
> > >> > with
> > >> > >> > multiple values per key. The key being the grouping key, and
> the
> > >> > values
> > >> > >> > consisting of one value per stream.
> > >> > >> >
> > >> > >> > What I like about Kyle's approach is that allows elegant
> > >> co-grouping
> > >> > of
> > >> > >> > multiple streams without having to worry about the number of
> > >> streams,
> > >> > >> and
> > >> > >> > avoids dealing with Tuple types or other generic interfaces
> that
> > >> could
> > >> > >> get
> > >> > >> > messy if we wanted to preserve all the value types in the
> > resulting
> > >> > >> > co-grouped stream.
> > >> > >> >
> > >> > >> > My only concern is that we only allow the cogroup + aggregate
> > >> combined
> > >> > >> > operation. This forces the user to build their own tuple
> > >> serialization
> > >> > >> > format if they want to preserve the individual input stream
> > values
> > >> as
> > >> > a
> > >> > >> > group. It also deviates quite a bit from our approach in
> > >> > KGroupedStream
> > >> > >> > which offers other operations, such as count and reduce, which
> > >> should
> > >> > >> also
> > >> > >> > be applicable to a co-grouped stream.
> > >> > >> >
> > >> > >> > Overall I still think this is a really useful addition, but I
> > feel
> > >> we
> > >> > >> > haven't spend much time trying to explore alternative DSLs that
> > >> could
> > >> > >> maybe
> > >> > >> > generalize better or match our existing syntax more closely.
> > >> > >> >
> > >> > >> > On Tue, May 9, 2017 at 8:08 AM Kyle Winkelman <
> > >> > winkelman.kyle@gmail.com
> > >> > >> >
> > >> > >> > wrote:
> > >> > >> >
> > >> > >> > > Eno, is there anyone else that is an expert in the kafka
> > streams
> > >> > realm
> > >> > >> > that
> > >> > >> > > I should reach out to for input?
> > >> > >> > >
> > >> > >> > > I believe Damian Guy is still planning on reviewing this more
> > in
> > >> > depth
> > >> > >> > so I
> > >> > >> > > will wait for his inputs before continuing.
> > >> > >> > >
> > >> > >> > > On May 9, 2017 7:30 AM, "Eno Thereska" <
> eno.thereska@gmail.com
> > >
> > >> > >> wrote:
> > >> > >> > >
> > >> > >> > > > Thanks Kyle, good arguments.
> > >> > >> > > >
> > >> > >> > > > Eno
> > >> > >> > > >
> > >> > >> > > > > On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > >> > >> winkelman.kyle@gmail.com
> > >> > >> > >
> > >> > >> > > > wrote:
> > >> > >> > > > >
> > >> > >> > > > > *- minor: could you add an exact example (similar to what
> > >> Jay’s
> > >> > >> > example
> > >> > >> > > > is,
> > >> > >> > > > > or like your Spark/Pig pointers had) to make this super
> > >> > concrete?*
> > >> > >> > > > > I have added a more concrete example to the KIP.
> > >> > >> > > > >
> > >> > >> > > > > *- my main concern is that we’re exposing this
> optimization
> > >> to
> > >> > the
> > >> > >> > DSL.
> > >> > >> > > > In
> > >> > >> > > > > an ideal world, an optimizer would take the existing DSL
> > and
> > >> do
> > >> > >> the
> > >> > >> > > right
> > >> > >> > > > > thing under the covers (create just one state store,
> > arrange
> > >> the
> > >> > >> > nodes
> > >> > >> > > > > etc). The original DSL had a bunch of small, composable
> > >> pieces
> > >> > >> > (group,
> > >> > >> > > > > aggregate, join) that this proposal groups together. I’d
> > >> like to
> > >> > >> hear
> > >> > >> > > > your
> > >> > >> > > > > thoughts on whether it’s possible to do this optimization
> > >> with
> > >> > the
> > >> > >> > > > current
> > >> > >> > > > > DSL, at the topology builder level.*
> > >> > >> > > > > You would have to make a lot of checks to understand if
> it
> > is
> > >> > even
> > >> > >> > > > possible
> > >> > >> > > > > to make this optimization:
> > >> > >> > > > > 1. Make sure they are all KTableKTableOuterJoins
> > >> > >> > > > > 2. None of the intermediate KTables are used for anything
> > >> else.
> > >> > >> > > > > 3. None of the intermediate stores are used. (This may be
> > >> > >> impossible
> > >> > >> > > > > especially if they use KafkaStreams#store after the
> > topology
> > >> has
> > >> > >> > > already
> > >> > >> > > > > been built.)
> > >> > >> > > > > You would then need to make decisions during the
> > >> optimization:
> > >> > >> > > > > 1. Your new initializer would the composite of all the
> > >> > individual
> > >> > >> > > > > initializers and the valueJoiners.
> > >> > >> > > > > 2. I am having a hard time thinking about how you would
> > turn
> > >> the
> > >> > >> > > > > aggregators and valueJoiners into an aggregator that
> would
> > >> work
> > >> > on
> > >> > >> > the
> > >> > >> > > > > final object, but this may be possible.
> > >> > >> > > > > 3. Which state store would you use? The ones declared
> would
> > >> be
> > >> > for
> > >> > >> > the
> > >> > >> > > > > aggregate values. None of the declared ones would be
> > >> guaranteed
> > >> > to
> > >> > >> > hold
> > >> > >> > > > the
> > >> > >> > > > > final object. This would mean you must created a new
> state
> > >> store
> > >> > >> and
> > >> > >> > > not
> > >> > >> > > > > created any of the declared ones.
> > >> > >> > > > >
> > >> > >> > > > > The main argument I have against it is even if it could
> be
> > >> done
> > >> > I
> > >> > >> > don't
> > >> > >> > > > > know that we would want to have this be an optimization
> in
> > >> the
> > >> > >> > > background
> > >> > >> > > > > because the user would still be required to think about
> all
> > >> of
> > >> > the
> > >> > >> > > > > intermediate values that they shouldn't need to worry
> about
> > >> if
> > >> > >> they
> > >> > >> > > only
> > >> > >> > > > > care about the final object.
> > >> > >> > > > >
> > >> > >> > > > > In my opinion cogroup is a common enough case that it
> > should
> > >> be
> > >> > >> part
> > >> > >> > of
> > >> > >> > > > the
> > >> > >> > > > > composable pieces (group, aggregate, join) because we
> want
> > to
> > >> > >> allow
> > >> > >> > > > people
> > >> > >> > > > > to join more than 2 or more streams in an easy way. Right
> > >> now I
> > >> > >> don't
> > >> > >> > > > think
> > >> > >> > > > > we give them ways of handling this use case easily.
> > >> > >> > > > >
> > >> > >> > > > > *-I think there will be scope for several such
> > optimizations
> > >> in
> > >> > >> the
> > >> > >> > > > future
> > >> > >> > > > > and perhaps at some point we need to think about
> decoupling
> > >> the
> > >> > >> 1:1
> > >> > >> > > > mapping
> > >> > >> > > > > from the DSL into the physical topology.*
> > >> > >> > > > > I would argue that cogroup is not just an optimization it
> > is
> > >> a
> > >> > new
> > >> > >> > way
> > >> > >> > > > for
> > >> > >> > > > > the users to look at accomplishing a problem that
> requires
> > >> > >> multiple
> > >> > >> > > > > streams. I may sound like a broken record but I don't
> think
> > >> > users
> > >> > >> > > should
> > >> > >> > > > > have to build the N-1 intermediate tables and deal with
> > their
> > >> > >> > > > initializers,
> > >> > >> > > > > serdes and stores if all they care about is the final
> > object.
> > >> > >> > > > > Now if for example someone uses cogroup but doesn't
> supply
> > >> > >> additional
> > >> > >> > > > > streams and aggregators this case is equivalent to a
> single
> > >> > >> grouped
> > >> > >> > > > stream
> > >> > >> > > > > making an aggregate call. This case is what I view an
> > >> > optimization
> > >> > >> > as,
> > >> > >> > > we
> > >> > >> > > > > could remove the KStreamCogroup and act as if there was
> > just
> > >> a
> > >> > >> call
> > >> > >> > to
> > >> > >> > > > > KGroupedStream#aggregate instead of calling
> > >> > >> KGroupedStream#cogroup.
> > >> > >> > (I
> > >> > >> > > > > would prefer to just write a warning saying that this is
> > not
> > >> how
> > >> > >> > > cogroup
> > >> > >> > > > is
> > >> > >> > > > > to be used.)
> > >> > >> > > > >
> > >> > >> > > > > Thanks,
> > >> > >> > > > > Kyle
> > >> > >> > > > >
> > >> > >> > > > > On Sun, May 7, 2017 at 5:41 AM, Eno Thereska <
> > >> > >> eno.thereska@gmail.com
> > >> > >> > >
> > >> > >> > > > wrote:
> > >> > >> > > > >
> > >> > >> > > > >> Hi Kyle,
> > >> > >> > > > >>
> > >> > >> > > > >> Thanks for the KIP again. A couple of comments:
> > >> > >> > > > >>
> > >> > >> > > > >> - minor: could you add an exact example (similar to what
> > >> Jay’s
> > >> > >> > example
> > >> > >> > > > is,
> > >> > >> > > > >> or like your Spark/Pig pointers had) to make this super
> > >> > concrete?
> > >> > >> > > > >>
> > >> > >> > > > >> - my main concern is that we’re exposing this
> optimization
> > >> to
> > >> > the
> > >> > >> > DSL.
> > >> > >> > > > In
> > >> > >> > > > >> an ideal world, an optimizer would take the existing DSL
> > >> and do
> > >> > >> the
> > >> > >> > > > right
> > >> > >> > > > >> thing under the covers (create just one state store,
> > arrange
> > >> > the
> > >> > >> > nodes
> > >> > >> > > > >> etc). The original DSL had a bunch of small, composable
> > >> pieces
> > >> > >> > (group,
> > >> > >> > > > >> aggregate, join) that this proposal groups together. I’d
> > >> like
> > >> > to
> > >> > >> > hear
> > >> > >> > > > your
> > >> > >> > > > >> thoughts on whether it’s possible to do this
> optimization
> > >> with
> > >> > >> the
> > >> > >> > > > current
> > >> > >> > > > >> DSL, at the topology builder level.
> > >> > >> > > > >>
> > >> > >> > > > >> I think there will be scope for several such
> optimizations
> > >> in
> > >> > the
> > >> > >> > > future
> > >> > >> > > > >> and perhaps at some point we need to think about
> > decoupling
> > >> the
> > >> > >> 1:1
> > >> > >> > > > mapping
> > >> > >> > > > >> from the DSL into the physical topology.
> > >> > >> > > > >>
> > >> > >> > > > >> Thanks
> > >> > >> > > > >> Eno
> > >> > >> > > > >>
> > >> > >> > > > >>> On May 5, 2017, at 4:39 PM, Jay Kreps <
> jay@confluent.io>
> > >> > wrote:
> > >> > >> > > > >>>
> > >> > >> > > > >>> I haven't digested the proposal but the use case is
> > pretty
> > >> > >> common.
> > >> > >> > An
> > >> > >> > > > >>> example would be the "customer 360" or "unified
> customer
> > >> > >> profile"
> > >> > >> > use
> > >> > >> > > > >> case
> > >> > >> > > > >>> we often use. In that use case you have a dozen systems
> > >> each
> > >> > of
> > >> > >> > which
> > >> > >> > > > has
> > >> > >> > > > >>> some information about your customer (account details,
> > >> > settings,
> > >> > >> > > > billing
> > >> > >> > > > >>> info, customer service contacts, purchase history,
> etc).
> > >> Your
> > >> > >> goal
> > >> > >> > is
> > >> > >> > > > to
> > >> > >> > > > >>> join/munge these into a single profile record for each
> > >> > customer
> > >> > >> > that
> > >> > >> > > > has
> > >> > >> > > > >>> all the relevant info in a usable form and is
> up-to-date
> > >> with
> > >> > >> all
> > >> > >> > the
> > >> > >> > > > >>> source systems. If you implement that with kstreams as
> a
> > >> > >> sequence
> > >> > >> > of
> > >> > >> > > > >> joins
> > >> > >> > > > >>> i think today we'd fully materialize N-1 intermediate
> > >> tables.
> > >> > >> But
> > >> > >> > > > clearly
> > >> > >> > > > >>> you only need a single stage to group all these things
> > that
> > >> > are
> > >> > >> > > already
> > >> > >> > > > >>> co-partitioned. A distributed database would do this
> > under
> > >> the
> > >> > >> > covers
> > >> > >> > > > >> which
> > >> > >> > > > >>> is arguably better (at least when it does the right
> > thing)
> > >> and
> > >> > >> > > perhaps
> > >> > >> > > > we
> > >> > >> > > > >>> could do the same thing but I'm not sure we know the
> > >> > >> partitioning
> > >> > >> > so
> > >> > >> > > we
> > >> > >> > > > >> may
> > >> > >> > > > >>> need an explicit cogroup command that impllies they are
> > >> > already
> > >> > >> > > > >>> co-partitioned.
> > >> > >> > > > >>>
> > >> > >> > > > >>> -Jay
> > >> > >> > > > >>>
> > >> > >> > > > >>> On Fri, May 5, 2017 at 5:56 AM, Kyle Winkelman <
> > >> > >> > > > winkelman.kyle@gmail.com
> > >> > >> > > > >>>
> > >> > >> > > > >>> wrote:
> > >> > >> > > > >>>
> > >> > >> > > > >>>> Yea thats a good way to look at it.
> > >> > >> > > > >>>> I have seen this type of functionality in a couple
> other
> > >> > >> platforms
> > >> > >> > > > like
> > >> > >> > > > >>>> spark and pig.
> > >> > >> > > > >>>> https://spark.apache.org/docs/0.6.2/api/core/spark/
> > >> > >> > > > >> PairRDDFunctions.html
> > >> > >> > > > >>>> https://www.tutorialspoint.com/apache_pig/apache_pig_
> > >> > >> > > > >> cogroup_operator.htm
> > >> > >> > > > >>>>
> > >> > >> > > > >>>>
> > >> > >> > > > >>>> On May 5, 2017 7:43 AM, "Damian Guy" <
> > >> damian.guy@gmail.com>
> > >> > >> > wrote:
> > >> > >> > > > >>>>
> > >> > >> > > > >>>>> Hi Kyle,
> > >> > >> > > > >>>>>
> > >> > >> > > > >>>>> If i'm reading this correctly it is like an N way
> outer
> > >> > join?
> > >> > >> So
> > >> > >> > an
> > >> > >> > > > >> input
> > >> > >> > > > >>>>> on any stream will always produce a new aggregated
> > value
> > >> -
> > >> > is
> > >> > >> > that
> > >> > >> > > > >>>> correct?
> > >> > >> > > > >>>>> Effectively, each Aggregator just looks up the
> current
> > >> > value,
> > >> > >> > > > >> aggregates
> > >> > >> > > > >>>>> and forwards the result.
> > >> > >> > > > >>>>> I need to look into it and think about it a bit more,
> > >> but it
> > >> > >> > seems
> > >> > >> > > > like
> > >> > >> > > > >>>> it
> > >> > >> > > > >>>>> could be a useful optimization.
> > >> > >> > > > >>>>>
> > >> > >> > > > >>>>> On Thu, 4 May 2017 at 23:21 Kyle Winkelman <
> > >> > >> > > winkelman.kyle@gmail.com
> > >> > >> > > > >
> > >> > >> > > > >>>>> wrote:
> > >> > >> > > > >>>>>
> > >> > >> > > > >>>>>> I sure can. I have added the following description
> to
> > my
> > >> > >> KIP. If
> > >> > >> > > > this
> > >> > >> > > > >>>>>> doesn't help let me know and I will take some more
> > time
> > >> to
> > >> > >> > build a
> > >> > >> > > > >>>>> diagram
> > >> > >> > > > >>>>>> and make more of a step by step description:
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> Example with Current API:
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> KTable<K, V1> table1 =
> > >> > >> > > > >>>>>> builder.stream("topic1").groupByKey().aggregate(
> > >> > initializer1
> > >> > >> ,
> > >> > >> > > > >>>>> aggregator1,
> > >> > >> > > > >>>>>> aggValueSerde1, storeName1);
> > >> > >> > > > >>>>>> KTable<K, V2> table2 =
> > >> > >> > > > >>>>>> builder.stream("topic2").groupByKey().aggregate(
> > >> > initializer2
> > >> > >> ,
> > >> > >> > > > >>>>> aggregator2,
> > >> > >> > > > >>>>>> aggValueSerde2, storeName2);
> > >> > >> > > > >>>>>> KTable<K, V3> table3 =
> > >> > >> > > > >>>>>> builder.stream("topic3").groupByKey().aggregate(
> > >> > initializer3
> > >> > >> ,
> > >> > >> > > > >>>>> aggregator3,
> > >> > >> > > > >>>>>> aggValueSerde3, storeName3);
> > >> > >> > > > >>>>>> KTable<K, CG> cogrouped = table1.outerJoin(table2,
> > >> > >> > > > >>>>>> joinerOneAndTwo).outerJoin(table3,
> > >> joinerOneTwoAndThree);
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> As you can see this creates 3 StateStores, requires
> 3
> > >> > >> > > initializers,
> > >> > >> > > > >>>> and 3
> > >> > >> > > > >>>>>> aggValueSerdes. This also adds the pressure to user
> to
> > >> > define
> > >> > >> > what
> > >> > >> > > > the
> > >> > >> > > > >>>>>> intermediate values are going to be (V1, V2, V3).
> They
> > >> are
> > >> > >> left
> > >> > >> > > > with a
> > >> > >> > > > >>>>>> couple choices, first to make V1, V2, and V3 all the
> > >> same
> > >> > as
> > >> > >> CG
> > >> > >> > > and
> > >> > >> > > > >> the
> > >> > >> > > > >>>>> two
> > >> > >> > > > >>>>>> joiners are more like mergers, or second make them
> > >> > >> intermediate
> > >> > >> > > > states
> > >> > >> > > > >>>>> such
> > >> > >> > > > >>>>>> as Topic1Map, Topic2Map, and Topic3Map and the
> joiners
> > >> use
> > >> > >> those
> > >> > >> > > to
> > >> > >> > > > >>>> build
> > >> > >> > > > >>>>>> the final aggregate CG value. This is something the
> > user
> > >> > >> could
> > >> > >> > > avoid
> > >> > >> > > > >>>>>> thinking about with this KIP.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> When a new input arrives lets say at "topic1" it
> will
> > >> first
> > >> > >> go
> > >> > >> > > > through
> > >> > >> > > > >>>> a
> > >> > >> > > > >>>>>> KStreamAggregate grabbing the current aggregate from
> > >> > >> storeName1.
> > >> > >> > > It
> > >> > >> > > > >>>> will
> > >> > >> > > > >>>>>> produce this in the form of the first intermediate
> > value
> > >> > and
> > >> > >> get
> > >> > >> > > > sent
> > >> > >> > > > >>>>>> through a KTableKTableOuterJoin where it will look
> up
> > >> the
> > >> > >> > current
> > >> > >> > > > >> value
> > >> > >> > > > >>>>> of
> > >> > >> > > > >>>>>> the key in storeName2. It will use the first joiner
> to
> > >> > >> calculate
> > >> > >> > > the
> > >> > >> > > > >>>>> second
> > >> > >> > > > >>>>>> intermediate value, which will go through an
> > additional
> > >> > >> > > > >>>>>> KTableKTableOuterJoin. Here it will look up the
> > current
> > >> > >> value of
> > >> > >> > > the
> > >> > >> > > > >>>> key
> > >> > >> > > > >>>>> in
> > >> > >> > > > >>>>>> storeName3 and use the second joiner to build the
> > final
> > >> > >> > aggregate
> > >> > >> > > > >>>> value.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> If you think through all possibilities for incoming
> > >> topics
> > >> > >> you
> > >> > >> > > will
> > >> > >> > > > >> see
> > >> > >> > > > >>>>>> that no matter which topic it comes in through all
> > three
> > >> > >> stores
> > >> > >> > > are
> > >> > >> > > > >>>>> queried
> > >> > >> > > > >>>>>> and all of the joiners must get used.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> Topology wise for N incoming streams this creates N
> > >> > >> > > > >>>>>> KStreamAggregates, 2*(N-1) KTableKTableOuterJoins,
> and
> > >> N-1
> > >> > >> > > > >>>>>> KTableKTableJoinMergers.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> Example with Proposed API:
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> KGroupedStream<K, V1> grouped1 =
> > >> builder.stream("topic1").
> > >> > >> > > > >>>> groupByKey();
> > >> > >> > > > >>>>>> KGroupedStream<K, V2> grouped2 =
> > >> builder.stream("topic2").
> > >> > >> > > > >>>> groupByKey();
> > >> > >> > > > >>>>>> KGroupedStream<K, V3> grouped3 =
> > >> builder.stream("topic3").
> > >> > >> > > > >>>> groupByKey();
> > >> > >> > > > >>>>>> KTable<K, CG> cogrouped =
> > grouped1.cogroup(initializer1,
> > >> > >> > > > aggregator1,
> > >> > >> > > > >>>>>> aggValueSerde1, storeName1)
> > >> > >> > > > >>>>>>       .cogroup(grouped2, aggregator2)
> > >> > >> > > > >>>>>>       .cogroup(grouped3, aggregator3)
> > >> > >> > > > >>>>>>       .aggregate();
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> As you can see this creates 1 StateStore, requires 1
> > >> > >> > initializer,
> > >> > >> > > > and
> > >> > >> > > > >> 1
> > >> > >> > > > >>>>>> aggValueSerde. The user no longer has to worry about
> > the
> > >> > >> > > > intermediate
> > >> > >> > > > >>>>>> values and the joiners. All they have to think about
> > is
> > >> how
> > >> > >> each
> > >> > >> > > > >> stream
> > >> > >> > > > >>>>>> impacts the creation of the final CG object.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> When a new input arrives lets say at "topic1" it
> will
> > >> first
> > >> > >> go
> > >> > >> > > > through
> > >> > >> > > > >>>> a
> > >> > >> > > > >>>>>> KStreamAggreagte and grab the current aggregate from
> > >> > >> storeName1.
> > >> > >> > > It
> > >> > >> > > > >>>> will
> > >> > >> > > > >>>>>> add its incoming object to the aggregate, update the
> > >> store
> > >> > >> and
> > >> > >> > > pass
> > >> > >> > > > >> the
> > >> > >> > > > >>>>> new
> > >> > >> > > > >>>>>> aggregate on. This new aggregate goes through the
> > >> > >> KStreamCogroup
> > >> > >> > > > which
> > >> > >> > > > >>>> is
> > >> > >> > > > >>>>>> pretty much just a pass through processor and you
> are
> > >> done.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> Topology wise for N incoming streams the new api
> will
> > >> only
> > >> > >> every
> > >> > >> > > > >>>> create N
> > >> > >> > > > >>>>>> KStreamAggregates and 1 KStreamCogroup.
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>> On Thu, May 4, 2017 at 4:42 PM, Matthias J. Sax <
> > >> > >> > > > >> matthias@confluent.io
> > >> > >> > > > >>>>>
> > >> > >> > > > >>>>>> wrote:
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>>> Kyle,
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>> thanks a lot for the KIP. Maybe I am a little slow,
> > >> but I
> > >> > >> could
> > >> > >> > > not
> > >> > >> > > > >>>>>>> follow completely. Could you maybe add a more
> > concrete
> > >> > >> example,
> > >> > >> > > > like
> > >> > >> > > > >>>> 3
> > >> > >> > > > >>>>>>> streams with 3 records each (plus expected result),
> > and
> > >> > show
> > >> > >> > the
> > >> > >> > > > >>>>>>> difference between current way to to implement it
> and
> > >> the
> > >> > >> > > proposed
> > >> > >> > > > >>>> API?
> > >> > >> > > > >>>>>>> This could also cover the internal processing to
> see
> > >> what
> > >> > >> store
> > >> > >> > > > calls
> > >> > >> > > > >>>>>>> would be required for both approaches etc.
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>> I think, it's pretty advanced stuff you propose,
> and
> > it
> > >> > >> would
> > >> > >> > > help
> > >> > >> > > > to
> > >> > >> > > > >>>>>>> understand it better.
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>> Thanks a lot!
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>> -Matthias
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>> On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> > >> > >> > > > >>>>>>>> I have made a pull request. It can be found here.
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> https://github.com/apache/kafka/pull/2975
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> I plan to write some more unit tests for my
> classes
> > >> and
> > >> > get
> > >> > >> > > around
> > >> > >> > > > >>>> to
> > >> > >> > > > >>>>>>>> writing documentation for the public api
> additions.
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> One thing I was curious about is during the
> > >> > >> > > > >>>>>>> KCogroupedStreamImpl#aggregate
> > >> > >> > > > >>>>>>>> method I pass null to the KGroupedStream#
> > >> > >> > repartitionIfRequired
> > >> > >> > > > >>>>> method.
> > >> > >> > > > >>>>>> I
> > >> > >> > > > >>>>>>>> can't supply the store name because if more than
> one
> > >> > >> grouped
> > >> > >> > > > stream
> > >> > >> > > > >>>>>>>> repartitions an error is thrown. Is there some
> name
> > >> that
> > >> > >> > someone
> > >> > >> > > > >>>> can
> > >> > >> > > > >>>>>>>> recommend or should I leave the null and allow it
> to
> > >> fall
> > >> > >> back
> > >> > >> > > to
> > >> > >> > > > >>>> the
> > >> > >> > > > >>>>>>>> KGroupedStream.name?
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> Should this be expanded to handle grouped tables?
> > This
> > >> > >> would
> > >> > >> > be
> > >> > >> > > > >>>>> pretty
> > >> > >> > > > >>>>>>> easy
> > >> > >> > > > >>>>>>>> for a normal aggregate but one allowing session
> > stores
> > >> > and
> > >> > >> > > > windowed
> > >> > >> > > > >>>>>>> stores
> > >> > >> > > > >>>>>>>> would required KTableSessionWindowAggregate and
> > >> > >> > > > >>>> KTableWindowAggregate
> > >> > >> > > > >>>>>>>> implementations.
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> Thanks,
> > >> > >> > > > >>>>>>>> Kyle
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>> On May 4, 2017 1:24 PM, "Eno Thereska" <
> > >> > >> > eno.thereska@gmail.com>
> > >> > >> > > > >>>>> wrote:
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>>> I’ll look as well asap, sorry, been swamped.
> > >> > >> > > > >>>>>>>>>
> > >> > >> > > > >>>>>>>>> Eno
> > >> > >> > > > >>>>>>>>>> On May 4, 2017, at 6:17 PM, Damian Guy <
> > >> > >> > damian.guy@gmail.com>
> > >> > >> > > > >>>>> wrote:
> > >> > >> > > > >>>>>>>>>>
> > >> > >> > > > >>>>>>>>>> Hi Kyle,
> > >> > >> > > > >>>>>>>>>>
> > >> > >> > > > >>>>>>>>>> Thanks for the KIP. I apologize that i haven't
> had
> > >> the
> > >> > >> > chance
> > >> > >> > > to
> > >> > >> > > > >>>>> look
> > >> > >> > > > >>>>>>> at
> > >> > >> > > > >>>>>>>>>> the KIP yet, but will schedule some time to look
> > >> into
> > >> > it
> > >> > >> > > > >>>> tomorrow.
> > >> > >> > > > >>>>>> For
> > >> > >> > > > >>>>>>>>> the
> > >> > >> > > > >>>>>>>>>> implementation, can you raise a PR against kafka
> > >> trunk
> > >> > >> and
> > >> > >> > > mark
> > >> > >> > > > >>>> it
> > >> > >> > > > >>>>> as
> > >> > >> > > > >>>>>>>>> WIP?
> > >> > >> > > > >>>>>>>>>> It will be easier to review what you have done.
> > >> > >> > > > >>>>>>>>>>
> > >> > >> > > > >>>>>>>>>> Thanks,
> > >> > >> > > > >>>>>>>>>> Damian
> > >> > >> > > > >>>>>>>>>>
> > >> > >> > > > >>>>>>>>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <
> > >> > >> > > > >>>>> winkelman.kyle@gmail.com
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>>>> wrote:
> > >> > >> > > > >>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>> I am replying to this in hopes it will draw
> some
> > >> > >> attention
> > >> > >> > to
> > >> > >> > > > my
> > >> > >> > > > >>>>> KIP
> > >> > >> > > > >>>>>>> as
> > >> > >> > > > >>>>>>>>> I
> > >> > >> > > > >>>>>>>>>>> haven't heard from anyone in a couple days.
> This
> > >> is my
> > >> > >> > first
> > >> > >> > > > KIP
> > >> > >> > > > >>>>> and
> > >> > >> > > > >>>>>>> my
> > >> > >> > > > >>>>>>>>>>> first large contribution to the project so I'm
> > >> sure I
> > >> > >> did
> > >> > >> > > > >>>>> something
> > >> > >> > > > >>>>>>>>> wrong.
> > >> > >> > > > >>>>>>>>>>> ;)
> > >> > >> > > > >>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <
> > >> > >> > > > >>>>> winkelman.kyle@gmail.com>
> > >> > >> > > > >>>>>>>>> wrote:
> > >> > >> > > > >>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>> Hello all,
> > >> > >> > > > >>>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>> I have created KIP-150 to facilitate
> discussion
> > >> about
> > >> > >> > adding
> > >> > >> > > > >>>>>> cogroup
> > >> > >> > > > >>>>>>> to
> > >> > >> > > > >>>>>>>>>>>> the streams DSL.
> > >> > >> > > > >>>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>> Please find the KIP here:
> > >> > >> > > > >>>>>>>>>>>> https://cwiki.apache.org/
> > >> > confluence/display/KAFKA/KIP-
> > >> > >> > > > >>>>>>>>>>>> 150+-+Kafka-Streams+Cogroup
> > >> > >> > > > >>>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>> Please find my initial implementation here:
> > >> > >> > > > >>>>>>>>>>>> https://github.com/KyleWinkelman/kafka
> > >> > >> > > > >>>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>> Thanks,
> > >> > >> > > > >>>>>>>>>>>> Kyle Winkelman
> > >> > >> > > > >>>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>>>
> > >> > >> > > > >>>>>>>>>
> > >> > >> > > > >>>>>>>>>
> > >> > >> > > > >>>>>>>>
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>>
> > >> > >> > > > >>>>>>
> > >> > >> > > > >>>>>
> > >> > >> > > > >>>>
> > >> > >> > > > >>
> > >> > >> > > > >>
> > >> > >> > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > -- Guozhang
> > >> > >
> > >> >
> > >> >
> > >> >
> > >> > --
> > >> > -- Guozhang
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -- Guozhang
> > >>
> > >
> >
>