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

2019-10-31 Thread Walker Carlson
I just sent out a call for a vote. I think everyone has had a good
discussion :). If there are any more thoughts I would love to hear them.

Walker

On Thu, Oct 31, 2019 at 10:36 AM Guozhang Wang  wrote:

> Hello Walker / Matthias,
>
> Thanks for your explanation. I can tell you've put a lot of thoughts into
> this already and it seems we cannot avoid adding new interfaces in any
> ways, so I will rest my arguments trying to reduce the number of
> first-class citizens in the Streams DSL :)
>
>
> Guozhang
>
>
> On Thu, Oct 31, 2019 at 12:50 AM Matthias J. Sax 
> wrote:
>
> > Interesting discussion. My personal take is as follows:
> >
> > (1) Co-group is not a special case of a multi-way KTable join, because
> > multiple record wit the same key from a single input stream should be
> > aggregated together and there are not update semantics. A co-group is
> > rather a muti-stream aggregation operation.
> >
> > (2) Semantically, non-windowed co-group is the same as (as laid out in
> > the KIP already):
> >
> > > KTable aggTable1 = stream1.groupByKey().aggregate(...);
> > > KTable aggTable2 = stream2.groupByKey().aggregate(...);
> > > ...
> > > KTable aggTableX = streamX.groupByKey().aggregate(...);
> > >
> > > KTable final = aggTable1.join(aggTable(2)join(aggTAbleX)
> >
> > However, I don't think that it would be possible for our optimizer to
> > rewrite one into the other, given what `Initializer`, `Aggregator`,
> > `ValueJoiner`, and `S a user would provide.
> >
> > Hence, `cogroup()` is a more efficient way to express the above using a
> > single store, instead of X stores are required above.
> >
> > For windowed co-group, especially session-windowed, it seems not
> > possible at all to rewrite co-group as independent aggregations followed
> > by joins. Note that sessions boundaries would be determined _after_ the
> > input streams are co-partitioned/merged in `cogroup()` and thus would be
> > different compare the the aggregate-join pattern.
> >
> > (3) For the current KIP writeup, I agree that adding `Named` to
> > `aggregate()` aligns best with the current API layout. I also don't
> > think that the overloads are a big issue, because they are spread out
> > over multiple helper interfaces.
> >
> >
> >
> > -Matthias
> >
> >
> >
> > On 10/29/19 10:38 AM, Walker Carlson wrote:
> > > Hi Gouzhang,
> > >
> > > I am not sure what you mean by "Fields from different streams are never
> > > aggregated together", this certainly can be the case but not the
> general
> > > rule. If we want to take care of the special cases where the key-sets
> are
> > > disjoint for each stream then they can be given no-op operators. This
> > would
> > > have the same effect as a stitching join as the function to update the
> > > store would have to be defined either way, even to just place it in.
> > >
> > > Now if we look at it from the other way, if we only specify the
> multiway
> > > join then the user will need to aggregate each stream. Then they must
> do
> > > the join which either will involve aggregators and value joiners or
> some
> > > questionable optimization that would rely on each aggregator defined
> for
> > a
> > > grouped stream meshing together. And this would all have to happen
> inside
> > > KStream.
> > >
> > > I do agree that there are optimizations that can be done on joining
> > > multiple tables per your example, in both cases whether it be a
> > "stitching
> > > join" or not. But I do not think the place to do it is in Streams. This
> > > could be relatively easy to accomplish. I think we save ourselves pain
> if
> > > we consider the tables and streams as separate cases, as aggregating
> > > multiple streams into one KTable can be done more efficiently than
> making
> > > multiple KTables and then joining them together. We may be able to get
> > > around this in the case of a stitching join but I am not sure how we
> > could
> > > do it safely otherwise.
> > >
> > > Walker
> > >
> > >
> > >
> > >
> > >
> > > On Mon, Oct 28, 2019 at 6:26 PM Guozhang Wang 
> > wrote:
> > >
> > >> Hi Walker,
> > >>
> > >> This is a good point about compatibility breakage while overloading
> the
> > >> existing classes; while reading John and your exchanges, I think I
> still
> > >> need to clarify the motivations a bit more:
> > >>
> > >> 1) Multiple streams need to be aggregated together, inputs are always
> > >> *KStreams* and end result is a *KTable*.
> > >> 2) Fields from different streams are never aggregated together, i.e.
> on
> > the
> > >> higher level it is more like a "stitching up" the fields and then
> doing
> > a
> > >> single aggregation.
> > >>
> > >> In this context, I agree with you that it is still a
> streams-aggregation
> > >> operator that we are trying to optimize (though its a multi-way), not
> a
> > >> multi-way table-table-join operator that we are tying to optimize
> here.
> > >>
> > >>
> > >> -
> > >>
> > >> But now taking a step back looking at it, I'm wondering, because of 2)

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

2019-10-31 Thread Guozhang Wang
Hello Walker / Matthias,

Thanks for your explanation. I can tell you've put a lot of thoughts into
this already and it seems we cannot avoid adding new interfaces in any
ways, so I will rest my arguments trying to reduce the number of
first-class citizens in the Streams DSL :)


Guozhang


On Thu, Oct 31, 2019 at 12:50 AM Matthias J. Sax 
wrote:

> Interesting discussion. My personal take is as follows:
>
> (1) Co-group is not a special case of a multi-way KTable join, because
> multiple record wit the same key from a single input stream should be
> aggregated together and there are not update semantics. A co-group is
> rather a muti-stream aggregation operation.
>
> (2) Semantically, non-windowed co-group is the same as (as laid out in
> the KIP already):
>
> > KTable aggTable1 = stream1.groupByKey().aggregate(...);
> > KTable aggTable2 = stream2.groupByKey().aggregate(...);
> > ...
> > KTable aggTableX = streamX.groupByKey().aggregate(...);
> >
> > KTable final = aggTable1.join(aggTable(2)join(aggTAbleX)
>
> However, I don't think that it would be possible for our optimizer to
> rewrite one into the other, given what `Initializer`, `Aggregator`,
> `ValueJoiner`, and `S a user would provide.
>
> Hence, `cogroup()` is a more efficient way to express the above using a
> single store, instead of X stores are required above.
>
> For windowed co-group, especially session-windowed, it seems not
> possible at all to rewrite co-group as independent aggregations followed
> by joins. Note that sessions boundaries would be determined _after_ the
> input streams are co-partitioned/merged in `cogroup()` and thus would be
> different compare the the aggregate-join pattern.
>
> (3) For the current KIP writeup, I agree that adding `Named` to
> `aggregate()` aligns best with the current API layout. I also don't
> think that the overloads are a big issue, because they are spread out
> over multiple helper interfaces.
>
>
>
> -Matthias
>
>
>
> On 10/29/19 10:38 AM, Walker Carlson wrote:
> > Hi Gouzhang,
> >
> > I am not sure what you mean by "Fields from different streams are never
> > aggregated together", this certainly can be the case but not the general
> > rule. If we want to take care of the special cases where the key-sets are
> > disjoint for each stream then they can be given no-op operators. This
> would
> > have the same effect as a stitching join as the function to update the
> > store would have to be defined either way, even to just place it in.
> >
> > Now if we look at it from the other way, if we only specify the multiway
> > join then the user will need to aggregate each stream. Then they must do
> > the join which either will involve aggregators and value joiners or some
> > questionable optimization that would rely on each aggregator defined for
> a
> > grouped stream meshing together. And this would all have to happen inside
> > KStream.
> >
> > I do agree that there are optimizations that can be done on joining
> > multiple tables per your example, in both cases whether it be a
> "stitching
> > join" or not. But I do not think the place to do it is in Streams. This
> > could be relatively easy to accomplish. I think we save ourselves pain if
> > we consider the tables and streams as separate cases, as aggregating
> > multiple streams into one KTable can be done more efficiently than making
> > multiple KTables and then joining them together. We may be able to get
> > around this in the case of a stitching join but I am not sure how we
> could
> > do it safely otherwise.
> >
> > Walker
> >
> >
> >
> >
> >
> > On Mon, Oct 28, 2019 at 6:26 PM Guozhang Wang 
> wrote:
> >
> >> Hi Walker,
> >>
> >> This is a good point about compatibility breakage while overloading the
> >> existing classes; while reading John and your exchanges, I think I still
> >> need to clarify the motivations a bit more:
> >>
> >> 1) Multiple streams need to be aggregated together, inputs are always
> >> *KStreams* and end result is a *KTable*.
> >> 2) Fields from different streams are never aggregated together, i.e. on
> the
> >> higher level it is more like a "stitching up" the fields and then doing
> a
> >> single aggregation.
> >>
> >> In this context, I agree with you that it is still a streams-aggregation
> >> operator that we are trying to optimize (though its a multi-way), not a
> >> multi-way table-table-join operator that we are tying to optimize here.
> >>
> >>
> >> -
> >>
> >> But now taking a step back looking at it, I'm wondering, because of 2)
> that
> >> all input streams do not have overlapping fields, we can generalize
> this to
> >> a broader scope. Consider this case for example:
> >>
> >> table1 = builder.table("topic1");
> >> table2 = builder.table("topic2");
> >> table3 = builder.table("topic3");
> >> table4 = table1.join(table2).join(table3);
> >>
> >> Suppose the join operations do not take out any fields or add any new
> >> fields, i.e. say table1 has fields A, table2 has fields B, and 

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

2019-10-31 Thread Matthias J. Sax
Interesting discussion. My personal take is as follows:

(1) Co-group is not a special case of a multi-way KTable join, because
multiple record wit the same key from a single input stream should be
aggregated together and there are not update semantics. A co-group is
rather a muti-stream aggregation operation.

(2) Semantically, non-windowed co-group is the same as (as laid out in
the KIP already):

> KTable aggTable1 = stream1.groupByKey().aggregate(...);
> KTable aggTable2 = stream2.groupByKey().aggregate(...);
> ...
> KTable aggTableX = streamX.groupByKey().aggregate(...);
> 
> KTable final = aggTable1.join(aggTable(2)join(aggTAbleX)

However, I don't think that it would be possible for our optimizer to
rewrite one into the other, given what `Initializer`, `Aggregator`,
`ValueJoiner`, and `S a user would provide.

Hence, `cogroup()` is a more efficient way to express the above using a
single store, instead of X stores are required above.

For windowed co-group, especially session-windowed, it seems not
possible at all to rewrite co-group as independent aggregations followed
by joins. Note that sessions boundaries would be determined _after_ the
input streams are co-partitioned/merged in `cogroup()` and thus would be
different compare the the aggregate-join pattern.

(3) For the current KIP writeup, I agree that adding `Named` to
`aggregate()` aligns best with the current API layout. I also don't
think that the overloads are a big issue, because they are spread out
over multiple helper interfaces.



-Matthias



On 10/29/19 10:38 AM, Walker Carlson wrote:
> Hi Gouzhang,
> 
> I am not sure what you mean by "Fields from different streams are never
> aggregated together", this certainly can be the case but not the general
> rule. If we want to take care of the special cases where the key-sets are
> disjoint for each stream then they can be given no-op operators. This would
> have the same effect as a stitching join as the function to update the
> store would have to be defined either way, even to just place it in.
> 
> Now if we look at it from the other way, if we only specify the multiway
> join then the user will need to aggregate each stream. Then they must do
> the join which either will involve aggregators and value joiners or some
> questionable optimization that would rely on each aggregator defined for a
> grouped stream meshing together. And this would all have to happen inside
> KStream.
> 
> I do agree that there are optimizations that can be done on joining
> multiple tables per your example, in both cases whether it be a "stitching
> join" or not. But I do not think the place to do it is in Streams. This
> could be relatively easy to accomplish. I think we save ourselves pain if
> we consider the tables and streams as separate cases, as aggregating
> multiple streams into one KTable can be done more efficiently than making
> multiple KTables and then joining them together. We may be able to get
> around this in the case of a stitching join but I am not sure how we could
> do it safely otherwise.
> 
> Walker
> 
> 
> 
> 
> 
> On Mon, Oct 28, 2019 at 6:26 PM Guozhang Wang  wrote:
> 
>> Hi Walker,
>>
>> This is a good point about compatibility breakage while overloading the
>> existing classes; while reading John and your exchanges, I think I still
>> need to clarify the motivations a bit more:
>>
>> 1) Multiple streams need to be aggregated together, inputs are always
>> *KStreams* and end result is a *KTable*.
>> 2) Fields from different streams are never aggregated together, i.e. on the
>> higher level it is more like a "stitching up" the fields and then doing a
>> single aggregation.
>>
>> In this context, I agree with you that it is still a streams-aggregation
>> operator that we are trying to optimize (though its a multi-way), not a
>> multi-way table-table-join operator that we are tying to optimize here.
>>
>>
>> -
>>
>> But now taking a step back looking at it, I'm wondering, because of 2) that
>> all input streams do not have overlapping fields, we can generalize this to
>> a broader scope. Consider this case for example:
>>
>> table1 = builder.table("topic1");
>> table2 = builder.table("topic2");
>> table3 = builder.table("topic3");
>> table4 = table1.join(table2).join(table3);
>>
>> Suppose the join operations do not take out any fields or add any new
>> fields, i.e. say table1 has fields A, table2 has fields B, and table2 has
>> fields C besides the key K, the table 4 has field {A, B, C} --- the join is
>> just "stitching up" the fields --- then the above topology can actually be
>> optimized in a similar way:
>>
>> * we only keep one materialized store in the form of K -> {A, B, C} as the
>> materialized store of the final join result of table4.
>> * when a record comes in from table1/2/3, just query the store on K, and
>> then update the corresponding A/B/C field and then writes back to the
>> store.
>>
>>
>> Then the above streams-aggregation operator can be 

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

2019-10-29 Thread Walker Carlson
Hi Gouzhang,

I am not sure what you mean by "Fields from different streams are never
aggregated together", this certainly can be the case but not the general
rule. If we want to take care of the special cases where the key-sets are
disjoint for each stream then they can be given no-op operators. This would
have the same effect as a stitching join as the function to update the
store would have to be defined either way, even to just place it in.

Now if we look at it from the other way, if we only specify the multiway
join then the user will need to aggregate each stream. Then they must do
the join which either will involve aggregators and value joiners or some
questionable optimization that would rely on each aggregator defined for a
grouped stream meshing together. And this would all have to happen inside
KStream.

I do agree that there are optimizations that can be done on joining
multiple tables per your example, in both cases whether it be a "stitching
join" or not. But I do not think the place to do it is in Streams. This
could be relatively easy to accomplish. I think we save ourselves pain if
we consider the tables and streams as separate cases, as aggregating
multiple streams into one KTable can be done more efficiently than making
multiple KTables and then joining them together. We may be able to get
around this in the case of a stitching join but I am not sure how we could
do it safely otherwise.

Walker





On Mon, Oct 28, 2019 at 6:26 PM Guozhang Wang  wrote:

> Hi Walker,
>
> This is a good point about compatibility breakage while overloading the
> existing classes; while reading John and your exchanges, I think I still
> need to clarify the motivations a bit more:
>
> 1) Multiple streams need to be aggregated together, inputs are always
> *KStreams* and end result is a *KTable*.
> 2) Fields from different streams are never aggregated together, i.e. on the
> higher level it is more like a "stitching up" the fields and then doing a
> single aggregation.
>
> In this context, I agree with you that it is still a streams-aggregation
> operator that we are trying to optimize (though its a multi-way), not a
> multi-way table-table-join operator that we are tying to optimize here.
>
>
> -
>
> But now taking a step back looking at it, I'm wondering, because of 2) that
> all input streams do not have overlapping fields, we can generalize this to
> a broader scope. Consider this case for example:
>
> table1 = builder.table("topic1");
> table2 = builder.table("topic2");
> table3 = builder.table("topic3");
> table4 = table1.join(table2).join(table3);
>
> Suppose the join operations do not take out any fields or add any new
> fields, i.e. say table1 has fields A, table2 has fields B, and table2 has
> fields C besides the key K, the table 4 has field {A, B, C} --- the join is
> just "stitching up" the fields --- then the above topology can actually be
> optimized in a similar way:
>
> * we only keep one materialized store in the form of K -> {A, B, C} as the
> materialized store of the final join result of table4.
> * when a record comes in from table1/2/3, just query the store on K, and
> then update the corresponding A/B/C field and then writes back to the
> store.
>
>
> Then the above streams-aggregation operator can be treated as a special
> case of this: you first aggregate separately on stream1/2/3 and generate
> table1/2/3, and then do this "stitching join", behind the scene we can
> optimize the topology to do exactly the co-group logic by updating the
> second bullet point above as an aggregation operator:
>
> * when a record comes in from *stream1/2/3*, just query the store on K, and
> then update the corresponding A/B/C field *with an aggregator *and then
> writes back to the store.
>
> -
>
> Personally I think this is better because with 1) larger applicable scope,
> and 2) without introducing new interfaces. But of course on the other side
> it requires us to do this optimization inside the Streams with some syntax
> hint from users (for example, users need to specify it is a "stitching
> join" such that all fields are still preserved in the join result). WDYT?
>
>
> Guozhang
>
>
> On Mon, Oct 28, 2019 at 4:20 PM Walker Carlson 
> wrote:
>
> > Hi John,
> >
> > Thank you for the background information. I think I understand your
> point.
> >
> > I believe that this could be fixed by making the motivation a little
> > clearer in the KIP.  I think that the motivation is when you have
> multiple
> > streams that need to aggregate together to form a single object the
> > current, non optimal, way to do this is through a multiway table join.
> This
> > is a little hacky. There is a slight but significant difference in these
> > cases, as in the null value handling you pointed out.
> >
> > For the example in the motivation, these tables were grouped streams so
> > they already dropped the null values. If we consider Cogroup sitting in
> the
> > same grey area that 

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

2019-10-28 Thread Guozhang Wang
Hi Walker,

This is a good point about compatibility breakage while overloading the
existing classes; while reading John and your exchanges, I think I still
need to clarify the motivations a bit more:

1) Multiple streams need to be aggregated together, inputs are always
*KStreams* and end result is a *KTable*.
2) Fields from different streams are never aggregated together, i.e. on the
higher level it is more like a "stitching up" the fields and then doing a
single aggregation.

In this context, I agree with you that it is still a streams-aggregation
operator that we are trying to optimize (though its a multi-way), not a
multi-way table-table-join operator that we are tying to optimize here.


-

But now taking a step back looking at it, I'm wondering, because of 2) that
all input streams do not have overlapping fields, we can generalize this to
a broader scope. Consider this case for example:

table1 = builder.table("topic1");
table2 = builder.table("topic2");
table3 = builder.table("topic3");
table4 = table1.join(table2).join(table3);

Suppose the join operations do not take out any fields or add any new
fields, i.e. say table1 has fields A, table2 has fields B, and table2 has
fields C besides the key K, the table 4 has field {A, B, C} --- the join is
just "stitching up" the fields --- then the above topology can actually be
optimized in a similar way:

* we only keep one materialized store in the form of K -> {A, B, C} as the
materialized store of the final join result of table4.
* when a record comes in from table1/2/3, just query the store on K, and
then update the corresponding A/B/C field and then writes back to the store.


Then the above streams-aggregation operator can be treated as a special
case of this: you first aggregate separately on stream1/2/3 and generate
table1/2/3, and then do this "stitching join", behind the scene we can
optimize the topology to do exactly the co-group logic by updating the
second bullet point above as an aggregation operator:

* when a record comes in from *stream1/2/3*, just query the store on K, and
then update the corresponding A/B/C field *with an aggregator *and then
writes back to the store.

-

Personally I think this is better because with 1) larger applicable scope,
and 2) without introducing new interfaces. But of course on the other side
it requires us to do this optimization inside the Streams with some syntax
hint from users (for example, users need to specify it is a "stitching
join" such that all fields are still preserved in the join result). WDYT?


Guozhang


On Mon, Oct 28, 2019 at 4:20 PM Walker Carlson 
wrote:

> Hi John,
>
> Thank you for the background information. I think I understand your point.
>
> I believe that this could be fixed by making the motivation a little
> clearer in the KIP.  I think that the motivation is when you have multiple
> streams that need to aggregate together to form a single object the
> current, non optimal, way to do this is through a multiway table join. This
> is a little hacky. There is a slight but significant difference in these
> cases, as in the null value handling you pointed out.
>
> For the example in the motivation, these tables were grouped streams so
> they already dropped the null values. If we consider Cogroup sitting in the
> same grey area that KGroupedStream does it should also behave this way. If
> you think about it that way it is more of an extension of KGroupedStream
> than KTable or KStream. Therefore I handle null values the same way
> KGroupedStream#aggregate does.
>
> Looking back I am not sure I understood you previous question fully at the
> time. I am sorry if my answer caused any confusion!
>
> Walker
>
> On Mon, Oct 28, 2019 at 2:49 PM John Roesler  wrote:
>
> > Hi Walker,
> >
> > Sorry for the delay in responding. Thanks for your response earlier.
> >
> > I think there might be a subtlety getting overlooked in considering
> > whether we're talking about streams versus tables in cogroup. As I'm
> > sure you know, Kafka Streams treats "stream" records as independent,
> > immutable, and opaque "facts", whereas we treat "table" records as a
> > sequence of updates to an entity identified by the record key (where
> > "update" means that each record's value represents the new state after
> > applying the update). For the most part, this is a clean separation,
> > but there is one special case where records with a "null" value are
> > interpreted as a tombstone in the table context (i.e., the record
> > indicates not that the new value of the entity is "null", but rather
> > that the entity has been deleted). In contrast, a record with a null
> > value in the stream context is _just_ a record with a null value; no
> > special semantics.
> >
> > The difficulty is that these two semantics clash at the stream/table
> > boundary. So, operations that convert streams to tables (like
> > KGroupedStream#aggregate) have to cope with ambiguity about whether to
> > 

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

2019-10-28 Thread Walker Carlson
Hi John,

Thank you for the background information. I think I understand your point.

I believe that this could be fixed by making the motivation a little
clearer in the KIP.  I think that the motivation is when you have multiple
streams that need to aggregate together to form a single object the
current, non optimal, way to do this is through a multiway table join. This
is a little hacky. There is a slight but significant difference in these
cases, as in the null value handling you pointed out.

For the example in the motivation, these tables were grouped streams so
they already dropped the null values. If we consider Cogroup sitting in the
same grey area that KGroupedStream does it should also behave this way. If
you think about it that way it is more of an extension of KGroupedStream
than KTable or KStream. Therefore I handle null values the same way
KGroupedStream#aggregate does.

Looking back I am not sure I understood you previous question fully at the
time. I am sorry if my answer caused any confusion!

Walker

On Mon, Oct 28, 2019 at 2:49 PM John Roesler  wrote:

> Hi Walker,
>
> Sorry for the delay in responding. Thanks for your response earlier.
>
> I think there might be a subtlety getting overlooked in considering
> whether we're talking about streams versus tables in cogroup. As I'm
> sure you know, Kafka Streams treats "stream" records as independent,
> immutable, and opaque "facts", whereas we treat "table" records as a
> sequence of updates to an entity identified by the record key (where
> "update" means that each record's value represents the new state after
> applying the update). For the most part, this is a clean separation,
> but there is one special case where records with a "null" value are
> interpreted as a tombstone in the table context (i.e., the record
> indicates not that the new value of the entity is "null", but rather
> that the entity has been deleted). In contrast, a record with a null
> value in the stream context is _just_ a record with a null value; no
> special semantics.
>
> The difficulty is that these two semantics clash at the stream/table
> boundary. So, operations that convert streams to tables (like
> KGroupedStream#aggregate) have to cope with ambiguity about whether to
> treat null values opaquely as null values, or as tombstones. I think
> I'll make a long story short and just say that this is a very, very
> complex issue. As a result (and as a bit of a punt), our
> KGroupedStream operations actually just discard null-valued records.
> This means that the following are _not_ equivalent programs:
>
> table1 =
>   stream("records")
> .filter(Record::isOk)
> .groupByKey()
> .aggregate(() -> new Record(), (key, value, agg) -> value)
> table2 =
>   table("record")
> .filter(Record::isOk)
>
> They look about the same, in that they'll both produce a
> KTable with the value being the latest state. But if a
> record is deleted in the upstream data (represented as a "null"
> value), that record would also be deleted in table2, but not in
> table1. Table1 would just perpetually contain the value immediately
> prior to the delete.
>
> This is why it makes me nervous to propose a new kind of _stream_
> operation ostensibly in order to solve a problem that presents itself
> in the _table_ context.
>
> If the goal is to provide a more efficient and convenient multi-way
> KTable join, I think it would be a good idea to consider an extension
> to the KTable API, not the KStream API. On the other hand, if this is
> not the goal, then the motivation of the KIP shouldn't say that it is.
> Instead, the KIP could provide some other motivation specifically for
> augmenting the KStream API.
>
> There is a third alternative that comes to mind, if you wish to
> resolve the long-standing dilemma around this semantic problem and
> specify in the KIP how exactly nulls are handled in this operator. But
> (although this seems on the face to be a good option), I think it
> might be a briarpatch. Even if we are able to reach a suitable design,
> we'd have to contend with the fact that it looks like the
> KGroupedStream API, but behaves differently.
>
> What do you think about all this?
>
> Thanks again for the KIP and the discussion!
> -John
>
> On Mon, Oct 28, 2019 at 3:32 PM Walker Carlson 
> wrote:
> >
> > Hi Gouzhang,
> >
> > Matthias and I did talk about overloading different a type of aggregate
> > methods in the cogroup that would take in the windows and returns a
> > windowed KTable. We decided that it would break too much with the current
> > pattern that was established in the normal KStream. We can revisit this
> if
> > you have a different opinion on the tradeoff.
> >
> > Walker
> >
> > On Mon, Oct 28, 2019 at 12:14 PM Guozhang Wang 
> wrote:
> >
> > > Hi Walker,
> > >
> > > On Fri, Oct 25, 2019 at 1:34 PM Walker Carlson 
> > > wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > > 1. I am familiar with the cogroup of spark, it is very similar to
> > > > their join 

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

2019-10-28 Thread John Roesler
Hi Walker,

Sorry for the delay in responding. Thanks for your response earlier.

I think there might be a subtlety getting overlooked in considering
whether we're talking about streams versus tables in cogroup. As I'm
sure you know, Kafka Streams treats "stream" records as independent,
immutable, and opaque "facts", whereas we treat "table" records as a
sequence of updates to an entity identified by the record key (where
"update" means that each record's value represents the new state after
applying the update). For the most part, this is a clean separation,
but there is one special case where records with a "null" value are
interpreted as a tombstone in the table context (i.e., the record
indicates not that the new value of the entity is "null", but rather
that the entity has been deleted). In contrast, a record with a null
value in the stream context is _just_ a record with a null value; no
special semantics.

The difficulty is that these two semantics clash at the stream/table
boundary. So, operations that convert streams to tables (like
KGroupedStream#aggregate) have to cope with ambiguity about whether to
treat null values opaquely as null values, or as tombstones. I think
I'll make a long story short and just say that this is a very, very
complex issue. As a result (and as a bit of a punt), our
KGroupedStream operations actually just discard null-valued records.
This means that the following are _not_ equivalent programs:

table1 =
  stream("records")
.filter(Record::isOk)
.groupByKey()
.aggregate(() -> new Record(), (key, value, agg) -> value)
table2 =
  table("record")
.filter(Record::isOk)

They look about the same, in that they'll both produce a
KTable with the value being the latest state. But if a
record is deleted in the upstream data (represented as a "null"
value), that record would also be deleted in table2, but not in
table1. Table1 would just perpetually contain the value immediately
prior to the delete.

This is why it makes me nervous to propose a new kind of _stream_
operation ostensibly in order to solve a problem that presents itself
in the _table_ context.

If the goal is to provide a more efficient and convenient multi-way
KTable join, I think it would be a good idea to consider an extension
to the KTable API, not the KStream API. On the other hand, if this is
not the goal, then the motivation of the KIP shouldn't say that it is.
Instead, the KIP could provide some other motivation specifically for
augmenting the KStream API.

There is a third alternative that comes to mind, if you wish to
resolve the long-standing dilemma around this semantic problem and
specify in the KIP how exactly nulls are handled in this operator. But
(although this seems on the face to be a good option), I think it
might be a briarpatch. Even if we are able to reach a suitable design,
we'd have to contend with the fact that it looks like the
KGroupedStream API, but behaves differently.

What do you think about all this?

Thanks again for the KIP and the discussion!
-John

On Mon, Oct 28, 2019 at 3:32 PM Walker Carlson  wrote:
>
> Hi Gouzhang,
>
> Matthias and I did talk about overloading different a type of aggregate
> methods in the cogroup that would take in the windows and returns a
> windowed KTable. We decided that it would break too much with the current
> pattern that was established in the normal KStream. We can revisit this if
> you have a different opinion on the tradeoff.
>
> Walker
>
> On Mon, Oct 28, 2019 at 12:14 PM Guozhang Wang  wrote:
>
> > Hi Walker,
> >
> > On Fri, Oct 25, 2019 at 1:34 PM Walker Carlson 
> > wrote:
> >
> > > Hi Guozhang,
> > >
> > > 1. I am familiar with the cogroup of spark, it is very similar to
> > > their join operator but instead it makes the values iterable. I think
> > that
> > > the use cases are different enough that it makes sense to specify the
> > > aggregator when we do.
> > >
> > > I like the idea of "absorb" and I think it could be useful. Although I do
> > > not think it is as intuitive.
> > >
> > > If we were to go that route we would either use more processors or do
> > > essentially the same thing but would have to store the information
> > > required to cogroup inside that KTable. I think this would violate some
> > > design principles. I would argue that we should consider adding absorb as
> > > well and auto re-write it to use cogroup.
> > >
> >
> > Yeah I think I agree with you about the internal design complexity with
> > "absorb"; I was primarily thinking if we can save ourselves from
> > introducing 3 more public classes with co-group. But it seems that without
> > introducing new classes there's no easy way for us to bound the scope of
> > co-grouping (like how many streams will be co-grouped together).
> >
> > LMK if you have some better ideas: generally speaking the less new public
> > interfaces we are introducing to fulfill a new feature the better, so I'd
> > push us to think twice and carefully before we go down the 

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

2019-10-28 Thread Walker Carlson
Hi Gouzhang,

Matthias and I did talk about overloading different a type of aggregate
methods in the cogroup that would take in the windows and returns a
windowed KTable. We decided that it would break too much with the current
pattern that was established in the normal KStream. We can revisit this if
you have a different opinion on the tradeoff.

Walker

On Mon, Oct 28, 2019 at 12:14 PM Guozhang Wang  wrote:

> Hi Walker,
>
> On Fri, Oct 25, 2019 at 1:34 PM Walker Carlson 
> wrote:
>
> > Hi Guozhang,
> >
> > 1. I am familiar with the cogroup of spark, it is very similar to
> > their join operator but instead it makes the values iterable. I think
> that
> > the use cases are different enough that it makes sense to specify the
> > aggregator when we do.
> >
> > I like the idea of "absorb" and I think it could be useful. Although I do
> > not think it is as intuitive.
> >
> > If we were to go that route we would either use more processors or do
> > essentially the same thing but would have to store the information
> > required to cogroup inside that KTable. I think this would violate some
> > design principles. I would argue that we should consider adding absorb as
> > well and auto re-write it to use cogroup.
> >
>
> Yeah I think I agree with you about the internal design complexity with
> "absorb"; I was primarily thinking if we can save ourselves from
> introducing 3 more public classes with co-group. But it seems that without
> introducing new classes there's no easy way for us to bound the scope of
> co-grouping (like how many streams will be co-grouped together).
>
> LMK if you have some better ideas: generally speaking the less new public
> interfaces we are introducing to fulfill a new feature the better, so I'd
> push us to think twice and carefully before we go down the route.
>
>
> >
> > 2. We have not considered this thought that would be a convenient
> > operation.
> >
> > 3. There is only one processor made. We are actually having the naming
> > conversation right now in the above thread
> >
> > 4, 5. fair points
> >
> > Walker
> >
> > On Fri, Oct 25, 2019 at 11:58 AM Guozhang Wang 
> wrote:
> >
> > > Hi Walker, thanks for the KIP! I made a pass on the writeup and have
> some
> > > comments below:
> > >
> > > Meta:
> > >
> > > 1. Syntax-wise, I'm wondering if we have compared our current proposal
> > with
> > > Spark's co-group syntax (I know they are targeting for different use
> > cases,
> > > but wondering if their syntax is closer to the join operator), what are
> > the
> > > syntax / semantics trade-off here?
> > >
> > > Just playing a devil's advocate here, if the main motivation is to
> > provide
> > > a more convienent multi-way join syntax, and in order to only have one
> > > materialized store we need to specify the final joined format at the
> > > beginning, then what about the following alternative (with the given
> > > example in your wiki page):
> > >
> > >
> > > KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
> > > KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> > > KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> > >
> > > KTable aggregated = grouped1.aggregate(initializer,
> materialized,
> > > aggregator1);
> > >
> > > aggregated.absorb(grouped2, aggregator2);  // I'm just using a random
> > name
> > > on top of my head here
> > >   .absorb(grouped3, aggregator3);
> > >
> > > In this way, we just add a new API to the KTable to "absorb" new
> streams
> > as
> > > aggregated results without needing to introduce new first citizen
> > classes.
> > >
> > > 2. From the DSL optimization, have we considered if we can auto
> re-write
> > > the user written old fashioned multi-join into this new DSL operator?
> > >
> > > 3. Although it is not needed for the wiki page itself, for internal
> > > implementation how many processor nodes would we create for the new
> > > operator, and how we can allow users to name them?
> > >
> > > Minor:
> > >
> > > 4. In "Public Interfaces", better add the templated generics to
> > > "KGroupedStream" as "KGroupedStream".
> > >
> > > 5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
> > > e.g. "TimeWindowed*CogroupedKStream*".
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > >
> > > On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax <
> matth...@confluent.io>
> > > wrote:
> > >
> > > > Walker,
> > > >
> > > > I am not sure if I can follow your argument. What do you exactly mean
> > by
> > > >
> > > > > I also
> > > > >> think that in this case it would be better to separate the 2
> option
> > > out
> > > > >> into separate overloads.
> > > >
> > > > Maybe you can give an example what method signature you have in mind?
> > > >
> > > > >> We could take a named parameter from upstream or add an extra
> naming
> > > > option
> > > > >> however I don't really see the advantage that would give.
> > > >
> > > > Are you familiar with KIP-307? Before KIP-307, KS generated all 

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

2019-10-28 Thread Walker Carlson
Thank you Bill,

I can get behind keeping the correct pattern and adding the Named object
into the aggregate. And adding something like Cogrouped Object is a bit
beyond the scope of this KIP.

I updated the KIP to include the changes to include the Named parameter and
some minor text fixes. If there are no other comments I believe we can take
this to a vote?

Thanks,
Walker

On Mon, Oct 28, 2019 at 8:28 AM Bill Bejeck  wrote:

> Hi Walker,
>
> Thanks for taking on KIP-150, the co-group will be very useful.
>
> Regarding the naming, IMHO, we should stick to the current pattern, and
> that is, we provide overloads with a "Named" operator for the
> "aggregate" methods (I believe those are the only ones that create a
> processor).
> Currently, that's what we have with various operators performing
> aggregation operations.
> I understand your concern about adding methods, but IMHO it would be very
> confusing to users why would break the current pattern we have at an
> arbitrary point in time.
>
> As for Sophie's suggestion of adding a "CoGrouped" configuration object, I
> can see the merits of that approach.  But IMHO, instead of doing so for one
> operation, maybe we should take a step back and consider refactoring to one
> configuration object overall (I believe John has suggested something
> similar in the past).  That is well beyond the scope of this KIP, but I
> think it would be better to stick with our current pattern and consider
> changes we can apply to the entire API in a later KIP.
>
> Just my 2 cents.
>
> Thanks,
> Bill
>
>
> On Fri, Oct 25, 2019 at 4:34 PM Walker Carlson 
> wrote:
>
> > Hi Guozhang,
> >
> > 1. I am familiar with the cogroup of spark, it is very similar to
> > their join operator but instead it makes the values iterable. I think
> that
> > the use cases are different enough that it makes sense to specify the
> > aggregator when we do.
> >
> > I like the idea of "absorb" and I think it could be useful. Although I do
> > not think it is as intuitive.
> >
> > If we were to go that route we would either use more processors or do
> > essentially the same thing but would have to store the information
> > required to cogroup inside that KTable. I think this would violate some
> > design principles. I would argue that we should consider adding absorb as
> > well and auto re-write it to use cogroup.
> >
> > 2. We have not considered this thought that would be a convenient
> > operation.
> >
> > 3. There is only one processor made. We are actually having the naming
> > conversation right now in the above thread
> >
> > 4, 5. fair points
> >
> > Walker
> >
> > On Fri, Oct 25, 2019 at 11:58 AM Guozhang Wang 
> wrote:
> >
> > > Hi Walker, thanks for the KIP! I made a pass on the writeup and have
> some
> > > comments below:
> > >
> > > Meta:
> > >
> > > 1. Syntax-wise, I'm wondering if we have compared our current proposal
> > with
> > > Spark's co-group syntax (I know they are targeting for different use
> > cases,
> > > but wondering if their syntax is closer to the join operator), what are
> > the
> > > syntax / semantics trade-off here?
> > >
> > > Just playing a devil's advocate here, if the main motivation is to
> > provide
> > > a more convienent multi-way join syntax, and in order to only have one
> > > materialized store we need to specify the final joined format at the
> > > beginning, then what about the following alternative (with the given
> > > example in your wiki page):
> > >
> > >
> > > KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
> > > KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> > > KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> > >
> > > KTable aggregated = grouped1.aggregate(initializer,
> materialized,
> > > aggregator1);
> > >
> > > aggregated.absorb(grouped2, aggregator2);  // I'm just using a random
> > name
> > > on top of my head here
> > >   .absorb(grouped3, aggregator3);
> > >
> > > In this way, we just add a new API to the KTable to "absorb" new
> streams
> > as
> > > aggregated results without needing to introduce new first citizen
> > classes.
> > >
> > > 2. From the DSL optimization, have we considered if we can auto
> re-write
> > > the user written old fashioned multi-join into this new DSL operator?
> > >
> > > 3. Although it is not needed for the wiki page itself, for internal
> > > implementation how many processor nodes would we create for the new
> > > operator, and how we can allow users to name them?
> > >
> > > Minor:
> > >
> > > 4. In "Public Interfaces", better add the templated generics to
> > > "KGroupedStream" as "KGroupedStream".
> > >
> > > 5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
> > > e.g. "TimeWindowed*CogroupedKStream*".
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > >
> > > On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax <
> matth...@confluent.io>
> > > wrote:
> > >
> > > > Walker,
> > > >
> > > > I am not sure if I can follow 

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

2019-10-28 Thread Guozhang Wang
Hi Walker,

On Fri, Oct 25, 2019 at 1:34 PM Walker Carlson 
wrote:

> Hi Guozhang,
>
> 1. I am familiar with the cogroup of spark, it is very similar to
> their join operator but instead it makes the values iterable. I think that
> the use cases are different enough that it makes sense to specify the
> aggregator when we do.
>
> I like the idea of "absorb" and I think it could be useful. Although I do
> not think it is as intuitive.
>
> If we were to go that route we would either use more processors or do
> essentially the same thing but would have to store the information
> required to cogroup inside that KTable. I think this would violate some
> design principles. I would argue that we should consider adding absorb as
> well and auto re-write it to use cogroup.
>

Yeah I think I agree with you about the internal design complexity with
"absorb"; I was primarily thinking if we can save ourselves from
introducing 3 more public classes with co-group. But it seems that without
introducing new classes there's no easy way for us to bound the scope of
co-grouping (like how many streams will be co-grouped together).

LMK if you have some better ideas: generally speaking the less new public
interfaces we are introducing to fulfill a new feature the better, so I'd
push us to think twice and carefully before we go down the route.


>
> 2. We have not considered this thought that would be a convenient
> operation.
>
> 3. There is only one processor made. We are actually having the naming
> conversation right now in the above thread
>
> 4, 5. fair points
>
> Walker
>
> On Fri, Oct 25, 2019 at 11:58 AM Guozhang Wang  wrote:
>
> > Hi Walker, thanks for the KIP! I made a pass on the writeup and have some
> > comments below:
> >
> > Meta:
> >
> > 1. Syntax-wise, I'm wondering if we have compared our current proposal
> with
> > Spark's co-group syntax (I know they are targeting for different use
> cases,
> > but wondering if their syntax is closer to the join operator), what are
> the
> > syntax / semantics trade-off here?
> >
> > Just playing a devil's advocate here, if the main motivation is to
> provide
> > a more convienent multi-way join syntax, and in order to only have one
> > materialized store we need to specify the final joined format at the
> > beginning, then what about the following alternative (with the given
> > example in your wiki page):
> >
> >
> > KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
> > KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> > KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> >
> > KTable aggregated = grouped1.aggregate(initializer, materialized,
> > aggregator1);
> >
> > aggregated.absorb(grouped2, aggregator2);  // I'm just using a random
> name
> > on top of my head here
> >   .absorb(grouped3, aggregator3);
> >
> > In this way, we just add a new API to the KTable to "absorb" new streams
> as
> > aggregated results without needing to introduce new first citizen
> classes.
> >
> > 2. From the DSL optimization, have we considered if we can auto re-write
> > the user written old fashioned multi-join into this new DSL operator?
> >
> > 3. Although it is not needed for the wiki page itself, for internal
> > implementation how many processor nodes would we create for the new
> > operator, and how we can allow users to name them?
> >
> > Minor:
> >
> > 4. In "Public Interfaces", better add the templated generics to
> > "KGroupedStream" as "KGroupedStream".
> >
> > 5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
> > e.g. "TimeWindowed*CogroupedKStream*".
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax 
> > wrote:
> >
> > > Walker,
> > >
> > > I am not sure if I can follow your argument. What do you exactly mean
> by
> > >
> > > > I also
> > > >> think that in this case it would be better to separate the 2 option
> > out
> > > >> into separate overloads.
> > >
> > > Maybe you can give an example what method signature you have in mind?
> > >
> > > >> We could take a named parameter from upstream or add an extra naming
> > > option
> > > >> however I don't really see the advantage that would give.
> > >
> > > Are you familiar with KIP-307? Before KIP-307, KS generated all names
> > > for all Processors. This makes it hard to reason about a Topology if
> > > it's getting complex. Adding `Named` to the new co-group operator would
> > > actually align with KIP-307.
> > >
> > > > It seems to go in
> > > >> the opposite direction from the cogroup configuration idea you
> > proposed.
> > >
> > > Can you elaborate? Not sure if I can follow.
> > >
> > >
> > >
> > > -Matthias
> > >
> > >
> > > On 10/24/19 10:20 AM, Walker Carlson wrote:
> > > > While I like the idea Sophie I don't think that it is necessary. I
> also
> > > > think that in this case it would be better to separate the 2 option
> out
> > > > into separate overloads.
> > > > We could take a named 

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

2019-10-28 Thread Bill Bejeck
Hi Walker,

Thanks for taking on KIP-150, the co-group will be very useful.

Regarding the naming, IMHO, we should stick to the current pattern, and
that is, we provide overloads with a "Named" operator for the
"aggregate" methods (I believe those are the only ones that create a
processor).
Currently, that's what we have with various operators performing
aggregation operations.
I understand your concern about adding methods, but IMHO it would be very
confusing to users why would break the current pattern we have at an
arbitrary point in time.

As for Sophie's suggestion of adding a "CoGrouped" configuration object, I
can see the merits of that approach.  But IMHO, instead of doing so for one
operation, maybe we should take a step back and consider refactoring to one
configuration object overall (I believe John has suggested something
similar in the past).  That is well beyond the scope of this KIP, but I
think it would be better to stick with our current pattern and consider
changes we can apply to the entire API in a later KIP.

Just my 2 cents.

Thanks,
Bill


On Fri, Oct 25, 2019 at 4:34 PM Walker Carlson 
wrote:

> Hi Guozhang,
>
> 1. I am familiar with the cogroup of spark, it is very similar to
> their join operator but instead it makes the values iterable. I think that
> the use cases are different enough that it makes sense to specify the
> aggregator when we do.
>
> I like the idea of "absorb" and I think it could be useful. Although I do
> not think it is as intuitive.
>
> If we were to go that route we would either use more processors or do
> essentially the same thing but would have to store the information
> required to cogroup inside that KTable. I think this would violate some
> design principles. I would argue that we should consider adding absorb as
> well and auto re-write it to use cogroup.
>
> 2. We have not considered this thought that would be a convenient
> operation.
>
> 3. There is only one processor made. We are actually having the naming
> conversation right now in the above thread
>
> 4, 5. fair points
>
> Walker
>
> On Fri, Oct 25, 2019 at 11:58 AM Guozhang Wang  wrote:
>
> > Hi Walker, thanks for the KIP! I made a pass on the writeup and have some
> > comments below:
> >
> > Meta:
> >
> > 1. Syntax-wise, I'm wondering if we have compared our current proposal
> with
> > Spark's co-group syntax (I know they are targeting for different use
> cases,
> > but wondering if their syntax is closer to the join operator), what are
> the
> > syntax / semantics trade-off here?
> >
> > Just playing a devil's advocate here, if the main motivation is to
> provide
> > a more convienent multi-way join syntax, and in order to only have one
> > materialized store we need to specify the final joined format at the
> > beginning, then what about the following alternative (with the given
> > example in your wiki page):
> >
> >
> > KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
> > KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> > KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> >
> > KTable aggregated = grouped1.aggregate(initializer, materialized,
> > aggregator1);
> >
> > aggregated.absorb(grouped2, aggregator2);  // I'm just using a random
> name
> > on top of my head here
> >   .absorb(grouped3, aggregator3);
> >
> > In this way, we just add a new API to the KTable to "absorb" new streams
> as
> > aggregated results without needing to introduce new first citizen
> classes.
> >
> > 2. From the DSL optimization, have we considered if we can auto re-write
> > the user written old fashioned multi-join into this new DSL operator?
> >
> > 3. Although it is not needed for the wiki page itself, for internal
> > implementation how many processor nodes would we create for the new
> > operator, and how we can allow users to name them?
> >
> > Minor:
> >
> > 4. In "Public Interfaces", better add the templated generics to
> > "KGroupedStream" as "KGroupedStream".
> >
> > 5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
> > e.g. "TimeWindowed*CogroupedKStream*".
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax 
> > wrote:
> >
> > > Walker,
> > >
> > > I am not sure if I can follow your argument. What do you exactly mean
> by
> > >
> > > > I also
> > > >> think that in this case it would be better to separate the 2 option
> > out
> > > >> into separate overloads.
> > >
> > > Maybe you can give an example what method signature you have in mind?
> > >
> > > >> We could take a named parameter from upstream or add an extra naming
> > > option
> > > >> however I don't really see the advantage that would give.
> > >
> > > Are you familiar with KIP-307? Before KIP-307, KS generated all names
> > > for all Processors. This makes it hard to reason about a Topology if
> > > it's getting complex. Adding `Named` to the new co-group operator would
> > > actually align with KIP-307.
> > >
> 

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

2019-10-25 Thread Walker Carlson
Hi Guozhang,

1. I am familiar with the cogroup of spark, it is very similar to
their join operator but instead it makes the values iterable. I think that
the use cases are different enough that it makes sense to specify the
aggregator when we do.

I like the idea of "absorb" and I think it could be useful. Although I do
not think it is as intuitive.

If we were to go that route we would either use more processors or do
essentially the same thing but would have to store the information
required to cogroup inside that KTable. I think this would violate some
design principles. I would argue that we should consider adding absorb as
well and auto re-write it to use cogroup.

2. We have not considered this thought that would be a convenient operation.

3. There is only one processor made. We are actually having the naming
conversation right now in the above thread

4, 5. fair points

Walker

On Fri, Oct 25, 2019 at 11:58 AM Guozhang Wang  wrote:

> Hi Walker, thanks for the KIP! I made a pass on the writeup and have some
> comments below:
>
> Meta:
>
> 1. Syntax-wise, I'm wondering if we have compared our current proposal with
> Spark's co-group syntax (I know they are targeting for different use cases,
> but wondering if their syntax is closer to the join operator), what are the
> syntax / semantics trade-off here?
>
> Just playing a devil's advocate here, if the main motivation is to provide
> a more convienent multi-way join syntax, and in order to only have one
> materialized store we need to specify the final joined format at the
> beginning, then what about the following alternative (with the given
> example in your wiki page):
>
>
> KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
> KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
>
> KTable aggregated = grouped1.aggregate(initializer, materialized,
> aggregator1);
>
> aggregated.absorb(grouped2, aggregator2);  // I'm just using a random name
> on top of my head here
>   .absorb(grouped3, aggregator3);
>
> In this way, we just add a new API to the KTable to "absorb" new streams as
> aggregated results without needing to introduce new first citizen classes.
>
> 2. From the DSL optimization, have we considered if we can auto re-write
> the user written old fashioned multi-join into this new DSL operator?
>
> 3. Although it is not needed for the wiki page itself, for internal
> implementation how many processor nodes would we create for the new
> operator, and how we can allow users to name them?
>
> Minor:
>
> 4. In "Public Interfaces", better add the templated generics to
> "KGroupedStream" as "KGroupedStream".
>
> 5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
> e.g. "TimeWindowed*CogroupedKStream*".
>
>
> Guozhang
>
>
>
>
> On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax 
> wrote:
>
> > Walker,
> >
> > I am not sure if I can follow your argument. What do you exactly mean by
> >
> > > I also
> > >> think that in this case it would be better to separate the 2 option
> out
> > >> into separate overloads.
> >
> > Maybe you can give an example what method signature you have in mind?
> >
> > >> We could take a named parameter from upstream or add an extra naming
> > option
> > >> however I don't really see the advantage that would give.
> >
> > Are you familiar with KIP-307? Before KIP-307, KS generated all names
> > for all Processors. This makes it hard to reason about a Topology if
> > it's getting complex. Adding `Named` to the new co-group operator would
> > actually align with KIP-307.
> >
> > > It seems to go in
> > >> the opposite direction from the cogroup configuration idea you
> proposed.
> >
> > Can you elaborate? Not sure if I can follow.
> >
> >
> >
> > -Matthias
> >
> >
> > On 10/24/19 10:20 AM, Walker Carlson wrote:
> > > While I like the idea Sophie I don't think that it is necessary. I also
> > > think that in this case it would be better to separate the 2 option out
> > > into separate overloads.
> > > We could take a named parameter from upstream or add an extra naming
> > option
> > > however I don't really see the advantage that would give. It seems to
> go
> > in
> > > the opposite direction from the cogroup configuration idea you
> proposed.
> > >
> > > John, I think it could be both. It depends on when you aggregate and
> what
> > > kind of data you have. In the example it is aggregating before joining,
> > > there are probably some cases where you could join before aggregating.
> > IMHO
> > > it would be easier to group all the streams together then perform the
> one
> > > operation that results in a single KTable.
> > >
> > >
> > >
> > > On Wed, Oct 23, 2019 at 9:58 PM Sophie Blee-Goldman <
> sop...@confluent.io
> > >
> > > wrote:
> > >
> > >>> I can personally not see any need to add other configuration
> > >> Famous last words?
> > >>
> > >> Just kidding, 95% confidence is more than enough 

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

2019-10-25 Thread Guozhang Wang
Hi Walker, thanks for the KIP! I made a pass on the writeup and have some
comments below:

Meta:

1. Syntax-wise, I'm wondering if we have compared our current proposal with
Spark's co-group syntax (I know they are targeting for different use cases,
but wondering if their syntax is closer to the join operator), what are the
syntax / semantics trade-off here?

Just playing a devil's advocate here, if the main motivation is to provide
a more convienent multi-way join syntax, and in order to only have one
materialized store we need to specify the final joined format at the
beginning, then what about the following alternative (with the given
example in your wiki page):


KGroupedStream grouped1 = builder.stream("topic1").groupByKey();
KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
KGroupedStream grouped3 = builder.stream("topic3").groupByKey();

KTable aggregated = grouped1.aggregate(initializer, materialized,
aggregator1);

aggregated.absorb(grouped2, aggregator2);  // I'm just using a random name
on top of my head here
  .absorb(grouped3, aggregator3);

In this way, we just add a new API to the KTable to "absorb" new streams as
aggregated results without needing to introduce new first citizen classes.

2. From the DSL optimization, have we considered if we can auto re-write
the user written old fashioned multi-join into this new DSL operator?

3. Although it is not needed for the wiki page itself, for internal
implementation how many processor nodes would we create for the new
operator, and how we can allow users to name them?

Minor:

4. In "Public Interfaces", better add the templated generics to
"KGroupedStream" as "KGroupedStream".

5. Naming wise, I'd suggest we keep the "K" together with Stream/Table,
e.g. "TimeWindowed*CogroupedKStream*".


Guozhang




On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax 
wrote:

> Walker,
>
> I am not sure if I can follow your argument. What do you exactly mean by
>
> > I also
> >> think that in this case it would be better to separate the 2 option out
> >> into separate overloads.
>
> Maybe you can give an example what method signature you have in mind?
>
> >> We could take a named parameter from upstream or add an extra naming
> option
> >> however I don't really see the advantage that would give.
>
> Are you familiar with KIP-307? Before KIP-307, KS generated all names
> for all Processors. This makes it hard to reason about a Topology if
> it's getting complex. Adding `Named` to the new co-group operator would
> actually align with KIP-307.
>
> > It seems to go in
> >> the opposite direction from the cogroup configuration idea you proposed.
>
> Can you elaborate? Not sure if I can follow.
>
>
>
> -Matthias
>
>
> On 10/24/19 10:20 AM, Walker Carlson wrote:
> > While I like the idea Sophie I don't think that it is necessary. I also
> > think that in this case it would be better to separate the 2 option out
> > into separate overloads.
> > We could take a named parameter from upstream or add an extra naming
> option
> > however I don't really see the advantage that would give. It seems to go
> in
> > the opposite direction from the cogroup configuration idea you proposed.
> >
> > John, I think it could be both. It depends on when you aggregate and what
> > kind of data you have. In the example it is aggregating before joining,
> > there are probably some cases where you could join before aggregating.
> IMHO
> > it would be easier to group all the streams together then perform the one
> > operation that results in a single KTable.
> >
> >
> >
> > On Wed, Oct 23, 2019 at 9:58 PM Sophie Blee-Goldman  >
> > wrote:
> >
> >>> I can personally not see any need to add other configuration
> >> Famous last words?
> >>
> >> Just kidding, 95% confidence is more than enough to  me (and better to
> >> optimize for current
> >> design than for hypothetical future changes).
> >>
> >> One last question I have then is about the operator/store/repartition
> >> naming -- seems like
> >> we can name the underlying store/changelog through the Materialized
> >> parameter, but do we
> >> also want to include an overload taking a Named parameter for the
> operator
> >> name (as in the
> >> KTable#join variations)?
> >>
> >> On Wed, Oct 23, 2019 at 5:14 PM Matthias J. Sax 
> >> wrote:
> >>
> >>> Interesting idea, Sophie.
> >>>
> >>> So far, we tried to reuse existing config objects and only add new ones
> >>> when needed to avoid creating "redundant" classes. This is of course a
> >>> reactive approach (with the drawback to deprecate stuff if we change
> it,
> >>> as you described).
> >>>
> >>> I can personally not see any need to add other configuration parameters
> >>> atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only
> a
> >>> single state store that we need to configure, and reusing
> `Materialized`
> >>> seems to be appropriate.
> >>>
> >>> Also note, that the `Initializer` is a mandatory parameter and not a
> >>> configuration and 

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

2019-10-25 Thread Walker Carlson
Matthias,

I am not familiar with KIP-307. After looking into it a bit I think I
understand better what Named is for and it answers some of my concerns. I
think that we should consider how many more methods it would make, there
would be 12 aggregate functions including timeWindowed and sessionWindowed.
If we were to add Named then Sophie's idea of using a Cogrouped object to
hold the parameters (materialized, named) makes a lot more sense.

Another option is that because there is only one processor made so we
probably do not need to have that option in the aggregate call. It would be
natural to add to the option to cogroup method in KGroupedStreams.

To Sophie's point earlier "(ie the first stream is joined as
stream.cogroup(Aggregator) while the subsequent ones are joined as
.cogroup(Stream, Aggregator) )", we could make it optional to have the
first stream joined stream.cogroup(Aggregator, Named)

On Thu, Oct 24, 2019 at 11:43 PM Matthias J. Sax 
wrote:

> Walker,
>
> I am not sure if I can follow your argument. What do you exactly mean by
>
> > I also
> >> think that in this case it would be better to separate the 2 option out
> >> into separate overloads.
>
> Maybe you can give an example what method signature you have in mind?
>
> >> We could take a named parameter from upstream or add an extra naming
> option
> >> however I don't really see the advantage that would give.
>
> Are you familiar with KIP-307? Before KIP-307, KS generated all names
> for all Processors. This makes it hard to reason about a Topology if
> it's getting complex. Adding `Named` to the new co-group operator would
> actually align with KIP-307.
>
> > It seems to go in
> >> the opposite direction from the cogroup configuration idea you proposed.
>
> Can you elaborate? Not sure if I can follow.
>
>
>
> -Matthias
>
>
> On 10/24/19 10:20 AM, Walker Carlson wrote:
> > While I like the idea Sophie I don't think that it is necessary. I also
> > think that in this case it would be better to separate the 2 option out
> > into separate overloads.
> > We could take a named parameter from upstream or add an extra naming
> option
> > however I don't really see the advantage that would give. It seems to go
> in
> > the opposite direction from the cogroup configuration idea you proposed.
> >
> > John, I think it could be both. It depends on when you aggregate and what
> > kind of data you have. In the example it is aggregating before joining,
> > there are probably some cases where you could join before aggregating.
> IMHO
> > it would be easier to group all the streams together then perform the one
> > operation that results in a single KTable.
> >
> >
> >
> > On Wed, Oct 23, 2019 at 9:58 PM Sophie Blee-Goldman  >
> > wrote:
> >
> >>> I can personally not see any need to add other configuration
> >> Famous last words?
> >>
> >> Just kidding, 95% confidence is more than enough to  me (and better to
> >> optimize for current
> >> design than for hypothetical future changes).
> >>
> >> One last question I have then is about the operator/store/repartition
> >> naming -- seems like
> >> we can name the underlying store/changelog through the Materialized
> >> parameter, but do we
> >> also want to include an overload taking a Named parameter for the
> operator
> >> name (as in the
> >> KTable#join variations)?
> >>
> >> On Wed, Oct 23, 2019 at 5:14 PM Matthias J. Sax 
> >> wrote:
> >>
> >>> Interesting idea, Sophie.
> >>>
> >>> So far, we tried to reuse existing config objects and only add new ones
> >>> when needed to avoid creating "redundant" classes. This is of course a
> >>> reactive approach (with the drawback to deprecate stuff if we change
> it,
> >>> as you described).
> >>>
> >>> I can personally not see any need to add other configuration parameters
> >>> atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only
> a
> >>> single state store that we need to configure, and reusing
> `Materialized`
> >>> seems to be appropriate.
> >>>
> >>> Also note, that the `Initializer` is a mandatory parameter and not a
> >>> configuration and should be passed directly, and not via a
> configuration
> >>> object.
> >>>
> >>>
> >>> -Matthias
> >>>
> >>> On 10/23/19 11:37 AM, Sophie Blee-Goldman wrote:
>  Thanks for the explanation, makes sense to me! As for the API, one
> >> other
>  thought I had is might we ever want or need to introduce any other
> >>> configs
>  or parameters in the future? Obviously that's difficult to say now (or
>  maybe the
>  answer seems obviously "no") but we seem to often end up needing to
> add
> >>> new
>  overloads and/or deprecate old ones as new features or requirements
> >> come
>  into
>  play.
> 
>  What do you (and others?) think about wrapping the config parameters
> >> (ie
>  everything
>  except the actual grouped streams) in a new config object? For
> example,
> >>> the
>  CogroupedStream#aggregate field could take a single Cogrouped object,
> 

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

2019-10-25 Thread Matthias J. Sax
Walker,

I am not sure if I can follow your argument. What do you exactly mean by

> I also
>> think that in this case it would be better to separate the 2 option out
>> into separate overloads.

Maybe you can give an example what method signature you have in mind?

>> We could take a named parameter from upstream or add an extra naming option
>> however I don't really see the advantage that would give.

Are you familiar with KIP-307? Before KIP-307, KS generated all names
for all Processors. This makes it hard to reason about a Topology if
it's getting complex. Adding `Named` to the new co-group operator would
actually align with KIP-307.

> It seems to go in
>> the opposite direction from the cogroup configuration idea you proposed.

Can you elaborate? Not sure if I can follow.



-Matthias


On 10/24/19 10:20 AM, Walker Carlson wrote:
> While I like the idea Sophie I don't think that it is necessary. I also
> think that in this case it would be better to separate the 2 option out
> into separate overloads.
> We could take a named parameter from upstream or add an extra naming option
> however I don't really see the advantage that would give. It seems to go in
> the opposite direction from the cogroup configuration idea you proposed.
> 
> John, I think it could be both. It depends on when you aggregate and what
> kind of data you have. In the example it is aggregating before joining,
> there are probably some cases where you could join before aggregating. IMHO
> it would be easier to group all the streams together then perform the one
> operation that results in a single KTable.
> 
> 
> 
> On Wed, Oct 23, 2019 at 9:58 PM Sophie Blee-Goldman 
> wrote:
> 
>>> I can personally not see any need to add other configuration
>> Famous last words?
>>
>> Just kidding, 95% confidence is more than enough to  me (and better to
>> optimize for current
>> design than for hypothetical future changes).
>>
>> One last question I have then is about the operator/store/repartition
>> naming -- seems like
>> we can name the underlying store/changelog through the Materialized
>> parameter, but do we
>> also want to include an overload taking a Named parameter for the operator
>> name (as in the
>> KTable#join variations)?
>>
>> On Wed, Oct 23, 2019 at 5:14 PM Matthias J. Sax 
>> wrote:
>>
>>> Interesting idea, Sophie.
>>>
>>> So far, we tried to reuse existing config objects and only add new ones
>>> when needed to avoid creating "redundant" classes. This is of course a
>>> reactive approach (with the drawback to deprecate stuff if we change it,
>>> as you described).
>>>
>>> I can personally not see any need to add other configuration parameters
>>> atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only a
>>> single state store that we need to configure, and reusing `Materialized`
>>> seems to be appropriate.
>>>
>>> Also note, that the `Initializer` is a mandatory parameter and not a
>>> configuration and should be passed directly, and not via a configuration
>>> object.
>>>
>>>
>>> -Matthias
>>>
>>> On 10/23/19 11:37 AM, Sophie Blee-Goldman wrote:
 Thanks for the explanation, makes sense to me! As for the API, one
>> other
 thought I had is might we ever want or need to introduce any other
>>> configs
 or parameters in the future? Obviously that's difficult to say now (or
 maybe the
 answer seems obviously "no") but we seem to often end up needing to add
>>> new
 overloads and/or deprecate old ones as new features or requirements
>> come
 into
 play.

 What do you (and others?) think about wrapping the config parameters
>> (ie
 everything
 except the actual grouped streams) in a new config object? For example,
>>> the
 CogroupedStream#aggregate field could take a single Cogrouped object,
 which itself would have an initializer and a materialized. If we ever
>>> need
 to add
 a new parameter, we can just add it to the Cogrouped class.

 Also, will the backing store be available for IQ if a Materialized is
 passed in?

 On Wed, Oct 23, 2019 at 10:49 AM Walker Carlson >>
 wrote:

> Hi Sophie,
>
> Thank you for your comments. As for the different methods signatures I
>>> have
> not really considered any other options but  while I do agree it is
> confusing, I don't see any obvious solutions. The problem is that the
> cogroup essentially pairs a group stream with an aggregator and when
>> it
>>> is
> first made the method is called on a groupedStream already. However
>> each
> subsequent stream-aggregator pair is added on to a cogroup stream so
>> it
> needs both arguments.
>
> For the second question you should not need a joiner. The idea is that
>>> you
> can collect many grouped streams with overlapping key spaces and any
>>> kind
> of value types. Once aggregated its value will be reduced into one
>> type.
> This is why you need only one initializer. Each 

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

2019-10-24 Thread Walker Carlson
While I like the idea Sophie I don't think that it is necessary. I also
think that in this case it would be better to separate the 2 option out
into separate overloads.
We could take a named parameter from upstream or add an extra naming option
however I don't really see the advantage that would give. It seems to go in
the opposite direction from the cogroup configuration idea you proposed.

John, I think it could be both. It depends on when you aggregate and what
kind of data you have. In the example it is aggregating before joining,
there are probably some cases where you could join before aggregating. IMHO
it would be easier to group all the streams together then perform the one
operation that results in a single KTable.



On Wed, Oct 23, 2019 at 9:58 PM Sophie Blee-Goldman 
wrote:

> > I can personally not see any need to add other configuration
> Famous last words?
>
> Just kidding, 95% confidence is more than enough to  me (and better to
> optimize for current
> design than for hypothetical future changes).
>
> One last question I have then is about the operator/store/repartition
> naming -- seems like
> we can name the underlying store/changelog through the Materialized
> parameter, but do we
> also want to include an overload taking a Named parameter for the operator
> name (as in the
> KTable#join variations)?
>
> On Wed, Oct 23, 2019 at 5:14 PM Matthias J. Sax 
> wrote:
>
> > Interesting idea, Sophie.
> >
> > So far, we tried to reuse existing config objects and only add new ones
> > when needed to avoid creating "redundant" classes. This is of course a
> > reactive approach (with the drawback to deprecate stuff if we change it,
> > as you described).
> >
> > I can personally not see any need to add other configuration parameters
> > atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only a
> > single state store that we need to configure, and reusing `Materialized`
> > seems to be appropriate.
> >
> > Also note, that the `Initializer` is a mandatory parameter and not a
> > configuration and should be passed directly, and not via a configuration
> > object.
> >
> >
> > -Matthias
> >
> > On 10/23/19 11:37 AM, Sophie Blee-Goldman wrote:
> > > Thanks for the explanation, makes sense to me! As for the API, one
> other
> > > thought I had is might we ever want or need to introduce any other
> > configs
> > > or parameters in the future? Obviously that's difficult to say now (or
> > > maybe the
> > > answer seems obviously "no") but we seem to often end up needing to add
> > new
> > > overloads and/or deprecate old ones as new features or requirements
> come
> > > into
> > > play.
> > >
> > > What do you (and others?) think about wrapping the config parameters
> (ie
> > > everything
> > > except the actual grouped streams) in a new config object? For example,
> > the
> > > CogroupedStream#aggregate field could take a single Cogrouped object,
> > > which itself would have an initializer and a materialized. If we ever
> > need
> > > to add
> > > a new parameter, we can just add it to the Cogrouped class.
> > >
> > > Also, will the backing store be available for IQ if a Materialized is
> > > passed in?
> > >
> > > On Wed, Oct 23, 2019 at 10:49 AM Walker Carlson  >
> > > wrote:
> > >
> > >> Hi Sophie,
> > >>
> > >> Thank you for your comments. As for the different methods signatures I
> > have
> > >> not really considered any other options but  while I do agree it is
> > >> confusing, I don't see any obvious solutions. The problem is that the
> > >> cogroup essentially pairs a group stream with an aggregator and when
> it
> > is
> > >> first made the method is called on a groupedStream already. However
> each
> > >> subsequent stream-aggregator pair is added on to a cogroup stream so
> it
> > >> needs both arguments.
> > >>
> > >> For the second question you should not need a joiner. The idea is that
> > you
> > >> can collect many grouped streams with overlapping key spaces and any
> > kind
> > >> of value types. Once aggregated its value will be reduced into one
> type.
> > >> This is why you need only one initializer. Each aggregator will need
> to
> > >> integrate the new value with the new object made in the initializer.
> > >> Does that make sense?
> > >>
> > >> This is a good question and I will include this explanation in the kip
> > as
> > >> well.
> > >>
> > >> Thanks,
> > >> Walker
> > >>
> > >> On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman <
> > sop...@confluent.io>
> > >> wrote:
> > >>
> > >>> Hey Walker,
> > >>>
> > >>> Thanks for the KIP! I have just a couple of questions:
> > >>>
> > >>> 1) It seems a little awkward to me that with the current API, we
> have a
> > >>> nearly identical
> > >>> "add stream to cogroup" method, except for the first which has a
> > >> different
> > >>> signature
> > >>> (ie the first stream is joined as stream.cogroup(Aggregator) while
> the
> > >>> subsequent ones
> > >>> are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it

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

2019-10-23 Thread Sophie Blee-Goldman
> I can personally not see any need to add other configuration
Famous last words?

Just kidding, 95% confidence is more than enough to  me (and better to
optimize for current
design than for hypothetical future changes).

One last question I have then is about the operator/store/repartition
naming -- seems like
we can name the underlying store/changelog through the Materialized
parameter, but do we
also want to include an overload taking a Named parameter for the operator
name (as in the
KTable#join variations)?

On Wed, Oct 23, 2019 at 5:14 PM Matthias J. Sax 
wrote:

> Interesting idea, Sophie.
>
> So far, we tried to reuse existing config objects and only add new ones
> when needed to avoid creating "redundant" classes. This is of course a
> reactive approach (with the drawback to deprecate stuff if we change it,
> as you described).
>
> I can personally not see any need to add other configuration parameters
> atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only a
> single state store that we need to configure, and reusing `Materialized`
> seems to be appropriate.
>
> Also note, that the `Initializer` is a mandatory parameter and not a
> configuration and should be passed directly, and not via a configuration
> object.
>
>
> -Matthias
>
> On 10/23/19 11:37 AM, Sophie Blee-Goldman wrote:
> > Thanks for the explanation, makes sense to me! As for the API, one other
> > thought I had is might we ever want or need to introduce any other
> configs
> > or parameters in the future? Obviously that's difficult to say now (or
> > maybe the
> > answer seems obviously "no") but we seem to often end up needing to add
> new
> > overloads and/or deprecate old ones as new features or requirements come
> > into
> > play.
> >
> > What do you (and others?) think about wrapping the config parameters (ie
> > everything
> > except the actual grouped streams) in a new config object? For example,
> the
> > CogroupedStream#aggregate field could take a single Cogrouped object,
> > which itself would have an initializer and a materialized. If we ever
> need
> > to add
> > a new parameter, we can just add it to the Cogrouped class.
> >
> > Also, will the backing store be available for IQ if a Materialized is
> > passed in?
> >
> > On Wed, Oct 23, 2019 at 10:49 AM Walker Carlson 
> > wrote:
> >
> >> Hi Sophie,
> >>
> >> Thank you for your comments. As for the different methods signatures I
> have
> >> not really considered any other options but  while I do agree it is
> >> confusing, I don't see any obvious solutions. The problem is that the
> >> cogroup essentially pairs a group stream with an aggregator and when it
> is
> >> first made the method is called on a groupedStream already. However each
> >> subsequent stream-aggregator pair is added on to a cogroup stream so it
> >> needs both arguments.
> >>
> >> For the second question you should not need a joiner. The idea is that
> you
> >> can collect many grouped streams with overlapping key spaces and any
> kind
> >> of value types. Once aggregated its value will be reduced into one type.
> >> This is why you need only one initializer. Each aggregator will need to
> >> integrate the new value with the new object made in the initializer.
> >> Does that make sense?
> >>
> >> This is a good question and I will include this explanation in the kip
> as
> >> well.
> >>
> >> Thanks,
> >> Walker
> >>
> >> On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman <
> sop...@confluent.io>
> >> wrote:
> >>
> >>> Hey Walker,
> >>>
> >>> Thanks for the KIP! I have just a couple of questions:
> >>>
> >>> 1) It seems a little awkward to me that with the current API, we have a
> >>> nearly identical
> >>> "add stream to cogroup" method, except for the first which has a
> >> different
> >>> signature
> >>> (ie the first stream is joined as stream.cogroup(Aggregator) while the
> >>> subsequent ones
> >>> are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it
> would
> >>> look like exactly,
> >>> but I was just wondering if you'd considered and/or rejected any
> >>> alternative APIs?
> >>>
> >>> 2) This might just be my lack of familiarity with "cogroup" as a
> concept,
> >>> but with the
> >>> current (non-optimal) API the user seems to have some control over how
> >>> exactly
> >>> the different streams are joined through the ValueJoiners. Would this
> new
> >>> cogroup
> >>> simply concatenate the values from the different cogroup streams, or
> >> could
> >>> users
> >>> potentially pass some kind of Joiner to the cogroup/aggregate methods?
> >> Or,
> >>> is the
> >>> whole point of cogroups that you no longer ever need to specify a
> Joiner?
> >>> If so, you
> >>> should add a short line to the KIP explaining that for those of us who
> >>> aren't fluent
> >>> in cogroup semantics :)
> >>>
> >>> Cheers,
> >>> Sophie
> >>>
> >>> On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
> >>> wrote:
> >>>
>  Good catch I updated that.
> 
>  I have made a PR for this KIP

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

2019-10-23 Thread Matthias J. Sax
Interesting idea, Sophie.

So far, we tried to reuse existing config objects and only add new ones
when needed to avoid creating "redundant" classes. This is of course a
reactive approach (with the drawback to deprecate stuff if we change it,
as you described).

I can personally not see any need to add other configuration parameters
atm, so it's a 95% obvious "no" IMHO. The final `aggregate()` has only a
single state store that we need to configure, and reusing `Materialized`
seems to be appropriate.

Also note, that the `Initializer` is a mandatory parameter and not a
configuration and should be passed directly, and not via a configuration
object.


-Matthias

On 10/23/19 11:37 AM, Sophie Blee-Goldman wrote:
> Thanks for the explanation, makes sense to me! As for the API, one other
> thought I had is might we ever want or need to introduce any other configs
> or parameters in the future? Obviously that's difficult to say now (or
> maybe the
> answer seems obviously "no") but we seem to often end up needing to add new
> overloads and/or deprecate old ones as new features or requirements come
> into
> play.
> 
> What do you (and others?) think about wrapping the config parameters (ie
> everything
> except the actual grouped streams) in a new config object? For example, the
> CogroupedStream#aggregate field could take a single Cogrouped object,
> which itself would have an initializer and a materialized. If we ever need
> to add
> a new parameter, we can just add it to the Cogrouped class.
> 
> Also, will the backing store be available for IQ if a Materialized is
> passed in?
> 
> On Wed, Oct 23, 2019 at 10:49 AM Walker Carlson 
> wrote:
> 
>> Hi Sophie,
>>
>> Thank you for your comments. As for the different methods signatures I have
>> not really considered any other options but  while I do agree it is
>> confusing, I don't see any obvious solutions. The problem is that the
>> cogroup essentially pairs a group stream with an aggregator and when it is
>> first made the method is called on a groupedStream already. However each
>> subsequent stream-aggregator pair is added on to a cogroup stream so it
>> needs both arguments.
>>
>> For the second question you should not need a joiner. The idea is that you
>> can collect many grouped streams with overlapping key spaces and any kind
>> of value types. Once aggregated its value will be reduced into one type.
>> This is why you need only one initializer. Each aggregator will need to
>> integrate the new value with the new object made in the initializer.
>> Does that make sense?
>>
>> This is a good question and I will include this explanation in the kip as
>> well.
>>
>> Thanks,
>> Walker
>>
>> On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman 
>> wrote:
>>
>>> Hey Walker,
>>>
>>> Thanks for the KIP! I have just a couple of questions:
>>>
>>> 1) It seems a little awkward to me that with the current API, we have a
>>> nearly identical
>>> "add stream to cogroup" method, except for the first which has a
>> different
>>> signature
>>> (ie the first stream is joined as stream.cogroup(Aggregator) while the
>>> subsequent ones
>>> are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it would
>>> look like exactly,
>>> but I was just wondering if you'd considered and/or rejected any
>>> alternative APIs?
>>>
>>> 2) This might just be my lack of familiarity with "cogroup" as a concept,
>>> but with the
>>> current (non-optimal) API the user seems to have some control over how
>>> exactly
>>> the different streams are joined through the ValueJoiners. Would this new
>>> cogroup
>>> simply concatenate the values from the different cogroup streams, or
>> could
>>> users
>>> potentially pass some kind of Joiner to the cogroup/aggregate methods?
>> Or,
>>> is the
>>> whole point of cogroups that you no longer ever need to specify a Joiner?
>>> If so, you
>>> should add a short line to the KIP explaining that for those of us who
>>> aren't fluent
>>> in cogroup semantics :)
>>>
>>> Cheers,
>>> Sophie
>>>
>>> On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
>>> wrote:
>>>
 Good catch I updated that.

 I have made a PR for this KIP

 I then am splitting it into 3 parts, first cogroup for a key-value
>> store
>>> (
 here ), then for a
 timeWindowedStore, and then a sessionWindowedStore + ensuring
>>> partitioning.

 Walker

 On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax <
>> matth...@confluent.io>
 wrote:

> Walker,
>
> thanks for picking up the KIP and reworking it for the changed API.
>
> Overall, the updated API suggestions make sense to me. The seem to
>>> align
> quite nicely with our current API design.
>
> One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> `Materialized` should be `V`, not `VR`?
>
>
> -Matthias
>
>
>
> On 10/14/19 2:57 PM, Walker Carlson wrote:
>>
>

>>>

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

2019-10-23 Thread Sophie Blee-Goldman
Thanks for the explanation, makes sense to me! As for the API, one other
thought I had is might we ever want or need to introduce any other configs
or parameters in the future? Obviously that's difficult to say now (or
maybe the
answer seems obviously "no") but we seem to often end up needing to add new
overloads and/or deprecate old ones as new features or requirements come
into
play.

What do you (and others?) think about wrapping the config parameters (ie
everything
except the actual grouped streams) in a new config object? For example, the
CogroupedStream#aggregate field could take a single Cogrouped object,
which itself would have an initializer and a materialized. If we ever need
to add
a new parameter, we can just add it to the Cogrouped class.

Also, will the backing store be available for IQ if a Materialized is
passed in?

On Wed, Oct 23, 2019 at 10:49 AM Walker Carlson 
wrote:

> Hi Sophie,
>
> Thank you for your comments. As for the different methods signatures I have
> not really considered any other options but  while I do agree it is
> confusing, I don't see any obvious solutions. The problem is that the
> cogroup essentially pairs a group stream with an aggregator and when it is
> first made the method is called on a groupedStream already. However each
> subsequent stream-aggregator pair is added on to a cogroup stream so it
> needs both arguments.
>
> For the second question you should not need a joiner. The idea is that you
> can collect many grouped streams with overlapping key spaces and any kind
> of value types. Once aggregated its value will be reduced into one type.
> This is why you need only one initializer. Each aggregator will need to
> integrate the new value with the new object made in the initializer.
> Does that make sense?
>
> This is a good question and I will include this explanation in the kip as
> well.
>
> Thanks,
> Walker
>
> On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman 
> wrote:
>
> > Hey Walker,
> >
> > Thanks for the KIP! I have just a couple of questions:
> >
> > 1) It seems a little awkward to me that with the current API, we have a
> > nearly identical
> > "add stream to cogroup" method, except for the first which has a
> different
> > signature
> > (ie the first stream is joined as stream.cogroup(Aggregator) while the
> > subsequent ones
> > are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it would
> > look like exactly,
> > but I was just wondering if you'd considered and/or rejected any
> > alternative APIs?
> >
> > 2) This might just be my lack of familiarity with "cogroup" as a concept,
> > but with the
> > current (non-optimal) API the user seems to have some control over how
> > exactly
> > the different streams are joined through the ValueJoiners. Would this new
> > cogroup
> > simply concatenate the values from the different cogroup streams, or
> could
> > users
> > potentially pass some kind of Joiner to the cogroup/aggregate methods?
> Or,
> > is the
> > whole point of cogroups that you no longer ever need to specify a Joiner?
> > If so, you
> > should add a short line to the KIP explaining that for those of us who
> > aren't fluent
> > in cogroup semantics :)
> >
> > Cheers,
> > Sophie
> >
> > On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
> > wrote:
> >
> > > Good catch I updated that.
> > >
> > > I have made a PR for this KIP
> > >
> > > I then am splitting it into 3 parts, first cogroup for a key-value
> store
> > (
> > > here ), then for a
> > > timeWindowedStore, and then a sessionWindowedStore + ensuring
> > partitioning.
> > >
> > > Walker
> > >
> > > On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax <
> matth...@confluent.io>
> > > wrote:
> > >
> > > > Walker,
> > > >
> > > > thanks for picking up the KIP and reworking it for the changed API.
> > > >
> > > > Overall, the updated API suggestions make sense to me. The seem to
> > align
> > > > quite nicely with our current API design.
> > > >
> > > > One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> > > > `Materialized` should be `V`, not `VR`?
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > >
> > > > On 10/14/19 2:57 PM, Walker Carlson wrote:
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > > here
> > > > > is a link
> > > > >
> > > > > On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson <
> > wcarl...@confluent.io>
> > > > > wrote:
> > > > >
> > > > >> Hello all,
> > > > >>
> > > > >> I have picked up and updated KIP-150. Due to changes to the
> project
> > > > since
> > > > >> KIP #150 was written there are a few items that need to be
> updated.
> > > > >>
> > > > >> First item that changed is the adoption of the Materialized
> > parameter.
> > > > >>
> > > > >> The second item is the WindowedBy. How the old KIP handles
> windowing
> > > is
> > > > >> that it overloads the aggregate function to take in a Window
> object
> > as
> > > > 

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

2019-10-23 Thread John Roesler
Hi Walker,

Thanks for picking up the KIP!

I hope you don't mind if I pile on a question also.

The motivation section depicts the current "non-optimal" case as a
multi-way KTable equi-join, but the proposed API looks like a
multi-way KStream windowed join+aggregate. My question is specifically
whether the intent is to improve the experience with KTables,
KStreams, or both.

Thanks,
-John

On Wed, Oct 23, 2019 at 12:49 PM Walker Carlson  wrote:
>
> Hi Sophie,
>
> Thank you for your comments. As for the different methods signatures I have
> not really considered any other options but  while I do agree it is
> confusing, I don't see any obvious solutions. The problem is that the
> cogroup essentially pairs a group stream with an aggregator and when it is
> first made the method is called on a groupedStream already. However each
> subsequent stream-aggregator pair is added on to a cogroup stream so it
> needs both arguments.
>
> For the second question you should not need a joiner. The idea is that you
> can collect many grouped streams with overlapping key spaces and any kind
> of value types. Once aggregated its value will be reduced into one type.
> This is why you need only one initializer. Each aggregator will need to
> integrate the new value with the new object made in the initializer.
> Does that make sense?
>
> This is a good question and I will include this explanation in the kip as
> well.
>
> Thanks,
> Walker
>
> On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman 
> wrote:
>
> > Hey Walker,
> >
> > Thanks for the KIP! I have just a couple of questions:
> >
> > 1) It seems a little awkward to me that with the current API, we have a
> > nearly identical
> > "add stream to cogroup" method, except for the first which has a different
> > signature
> > (ie the first stream is joined as stream.cogroup(Aggregator) while the
> > subsequent ones
> > are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it would
> > look like exactly,
> > but I was just wondering if you'd considered and/or rejected any
> > alternative APIs?
> >
> > 2) This might just be my lack of familiarity with "cogroup" as a concept,
> > but with the
> > current (non-optimal) API the user seems to have some control over how
> > exactly
> > the different streams are joined through the ValueJoiners. Would this new
> > cogroup
> > simply concatenate the values from the different cogroup streams, or could
> > users
> > potentially pass some kind of Joiner to the cogroup/aggregate methods? Or,
> > is the
> > whole point of cogroups that you no longer ever need to specify a Joiner?
> > If so, you
> > should add a short line to the KIP explaining that for those of us who
> > aren't fluent
> > in cogroup semantics :)
> >
> > Cheers,
> > Sophie
> >
> > On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
> > wrote:
> >
> > > Good catch I updated that.
> > >
> > > I have made a PR for this KIP
> > >
> > > I then am splitting it into 3 parts, first cogroup for a key-value store
> > (
> > > here ), then for a
> > > timeWindowedStore, and then a sessionWindowedStore + ensuring
> > partitioning.
> > >
> > > Walker
> > >
> > > On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax 
> > > wrote:
> > >
> > > > Walker,
> > > >
> > > > thanks for picking up the KIP and reworking it for the changed API.
> > > >
> > > > Overall, the updated API suggestions make sense to me. The seem to
> > align
> > > > quite nicely with our current API design.
> > > >
> > > > One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> > > > `Materialized` should be `V`, not `VR`?
> > > >
> > > >
> > > > -Matthias
> > > >
> > > >
> > > >
> > > > On 10/14/19 2:57 PM, Walker Carlson wrote:
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > > here
> > > > > is a link
> > > > >
> > > > > On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson <
> > wcarl...@confluent.io>
> > > > > wrote:
> > > > >
> > > > >> Hello all,
> > > > >>
> > > > >> I have picked up and updated KIP-150. Due to changes to the project
> > > > since
> > > > >> KIP #150 was written there are a few items that need to be updated.
> > > > >>
> > > > >> First item that changed is the adoption of the Materialized
> > parameter.
> > > > >>
> > > > >> The second item is the WindowedBy. How the old KIP handles windowing
> > > is
> > > > >> that it overloads the aggregate function to take in a Window object
> > as
> > > > well
> > > > >> as the other parameters. The current practice to window
> > > grouped-streams
> > > > is
> > > > >> to call windowedBy and receive a windowed stream object. The
> > existing
> > > > >> interface for a windowed stream made from a grouped stream will not
> > > work
> > > > >> for cogrouped streams. Hence, we have to make new interfaces for
> > > > cogrouped
> > > > >> windowed streams.
> > > > >>
> > > > >> Please take a look, I would like to hear your feedback,
> > > > >>
> 

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

2019-10-23 Thread Walker Carlson
Hi Sophie,

Thank you for your comments. As for the different methods signatures I have
not really considered any other options but  while I do agree it is
confusing, I don't see any obvious solutions. The problem is that the
cogroup essentially pairs a group stream with an aggregator and when it is
first made the method is called on a groupedStream already. However each
subsequent stream-aggregator pair is added on to a cogroup stream so it
needs both arguments.

For the second question you should not need a joiner. The idea is that you
can collect many grouped streams with overlapping key spaces and any kind
of value types. Once aggregated its value will be reduced into one type.
This is why you need only one initializer. Each aggregator will need to
integrate the new value with the new object made in the initializer.
Does that make sense?

This is a good question and I will include this explanation in the kip as
well.

Thanks,
Walker

On Tue, Oct 22, 2019 at 8:59 PM Sophie Blee-Goldman 
wrote:

> Hey Walker,
>
> Thanks for the KIP! I have just a couple of questions:
>
> 1) It seems a little awkward to me that with the current API, we have a
> nearly identical
> "add stream to cogroup" method, except for the first which has a different
> signature
> (ie the first stream is joined as stream.cogroup(Aggregator) while the
> subsequent ones
> are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it would
> look like exactly,
> but I was just wondering if you'd considered and/or rejected any
> alternative APIs?
>
> 2) This might just be my lack of familiarity with "cogroup" as a concept,
> but with the
> current (non-optimal) API the user seems to have some control over how
> exactly
> the different streams are joined through the ValueJoiners. Would this new
> cogroup
> simply concatenate the values from the different cogroup streams, or could
> users
> potentially pass some kind of Joiner to the cogroup/aggregate methods? Or,
> is the
> whole point of cogroups that you no longer ever need to specify a Joiner?
> If so, you
> should add a short line to the KIP explaining that for those of us who
> aren't fluent
> in cogroup semantics :)
>
> Cheers,
> Sophie
>
> On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
> wrote:
>
> > Good catch I updated that.
> >
> > I have made a PR for this KIP
> >
> > I then am splitting it into 3 parts, first cogroup for a key-value store
> (
> > here ), then for a
> > timeWindowedStore, and then a sessionWindowedStore + ensuring
> partitioning.
> >
> > Walker
> >
> > On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax 
> > wrote:
> >
> > > Walker,
> > >
> > > thanks for picking up the KIP and reworking it for the changed API.
> > >
> > > Overall, the updated API suggestions make sense to me. The seem to
> align
> > > quite nicely with our current API design.
> > >
> > > One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> > > `Materialized` should be `V`, not `VR`?
> > >
> > >
> > > -Matthias
> > >
> > >
> > >
> > > On 10/14/19 2:57 PM, Walker Carlson wrote:
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > > here
> > > > is a link
> > > >
> > > > On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson <
> wcarl...@confluent.io>
> > > > wrote:
> > > >
> > > >> Hello all,
> > > >>
> > > >> I have picked up and updated KIP-150. Due to changes to the project
> > > since
> > > >> KIP #150 was written there are a few items that need to be updated.
> > > >>
> > > >> First item that changed is the adoption of the Materialized
> parameter.
> > > >>
> > > >> The second item is the WindowedBy. How the old KIP handles windowing
> > is
> > > >> that it overloads the aggregate function to take in a Window object
> as
> > > well
> > > >> as the other parameters. The current practice to window
> > grouped-streams
> > > is
> > > >> to call windowedBy and receive a windowed stream object. The
> existing
> > > >> interface for a windowed stream made from a grouped stream will not
> > work
> > > >> for cogrouped streams. Hence, we have to make new interfaces for
> > > cogrouped
> > > >> windowed streams.
> > > >>
> > > >> Please take a look, I would like to hear your feedback,
> > > >>
> > > >> Walker
> > > >>
> > > >
> > >
> > >
> >
>


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

2019-10-22 Thread Sophie Blee-Goldman
Hey Walker,

Thanks for the KIP! I have just a couple of questions:

1) It seems a little awkward to me that with the current API, we have a
nearly identical
"add stream to cogroup" method, except for the first which has a different
signature
(ie the first stream is joined as stream.cogroup(Aggregator) while the
subsequent ones
are joined as .cogroup(Stream, Aggregator) ). I'm not sure what it would
look like exactly,
but I was just wondering if you'd considered and/or rejected any
alternative APIs?

2) This might just be my lack of familiarity with "cogroup" as a concept,
but with the
current (non-optimal) API the user seems to have some control over how
exactly
the different streams are joined through the ValueJoiners. Would this new
cogroup
simply concatenate the values from the different cogroup streams, or could
users
potentially pass some kind of Joiner to the cogroup/aggregate methods? Or,
is the
whole point of cogroups that you no longer ever need to specify a Joiner?
If so, you
should add a short line to the KIP explaining that for those of us who
aren't fluent
in cogroup semantics :)

Cheers,
Sophie

On Thu, Oct 17, 2019 at 3:06 PM Walker Carlson 
wrote:

> Good catch I updated that.
>
> I have made a PR for this KIP
>
> I then am splitting it into 3 parts, first cogroup for a key-value store (
> here ), then for a
> timeWindowedStore, and then a sessionWindowedStore + ensuring partitioning.
>
> Walker
>
> On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax 
> wrote:
>
> > Walker,
> >
> > thanks for picking up the KIP and reworking it for the changed API.
> >
> > Overall, the updated API suggestions make sense to me. The seem to align
> > quite nicely with our current API design.
> >
> > One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> > `Materialized` should be `V`, not `VR`?
> >
> >
> > -Matthias
> >
> >
> >
> > On 10/14/19 2:57 PM, Walker Carlson wrote:
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > > here
> > > is a link
> > >
> > > On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson 
> > > wrote:
> > >
> > >> Hello all,
> > >>
> > >> I have picked up and updated KIP-150. Due to changes to the project
> > since
> > >> KIP #150 was written there are a few items that need to be updated.
> > >>
> > >> First item that changed is the adoption of the Materialized parameter.
> > >>
> > >> The second item is the WindowedBy. How the old KIP handles windowing
> is
> > >> that it overloads the aggregate function to take in a Window object as
> > well
> > >> as the other parameters. The current practice to window
> grouped-streams
> > is
> > >> to call windowedBy and receive a windowed stream object. The existing
> > >> interface for a windowed stream made from a grouped stream will not
> work
> > >> for cogrouped streams. Hence, we have to make new interfaces for
> > cogrouped
> > >> windowed streams.
> > >>
> > >> Please take a look, I would like to hear your feedback,
> > >>
> > >> Walker
> > >>
> > >
> >
> >
>


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

2019-10-17 Thread Walker Carlson
Good catch I updated that.

I have made a PR for this KIP

I then am splitting it into 3 parts, first cogroup for a key-value store (
here ), then for a
timeWindowedStore, and then a sessionWindowedStore + ensuring partitioning.

Walker

On Tue, Oct 15, 2019 at 12:47 PM Matthias J. Sax 
wrote:

> Walker,
>
> thanks for picking up the KIP and reworking it for the changed API.
>
> Overall, the updated API suggestions make sense to me. The seem to align
> quite nicely with our current API design.
>
> One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
> `Materialized` should be `V`, not `VR`?
>
>
> -Matthias
>
>
>
> On 10/14/19 2:57 PM, Walker Carlson wrote:
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> > here
> > is a link
> >
> > On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson 
> > wrote:
> >
> >> Hello all,
> >>
> >> I have picked up and updated KIP-150. Due to changes to the project
> since
> >> KIP #150 was written there are a few items that need to be updated.
> >>
> >> First item that changed is the adoption of the Materialized parameter.
> >>
> >> The second item is the WindowedBy. How the old KIP handles windowing is
> >> that it overloads the aggregate function to take in a Window object as
> well
> >> as the other parameters. The current practice to window grouped-streams
> is
> >> to call windowedBy and receive a windowed stream object. The existing
> >> interface for a windowed stream made from a grouped stream will not work
> >> for cogrouped streams. Hence, we have to make new interfaces for
> cogrouped
> >> windowed streams.
> >>
> >> Please take a look, I would like to hear your feedback,
> >>
> >> Walker
> >>
> >
>
>


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

2019-10-15 Thread Matthias J. Sax
Walker,

thanks for picking up the KIP and reworking it for the changed API.

Overall, the updated API suggestions make sense to me. The seem to align
quite nicely with our current API design.

One nit: In `CogroupedKStream#aggregate(...)` the type parameter of
`Materialized` should be `V`, not `VR`?


-Matthias



On 10/14/19 2:57 PM, Walker Carlson wrote:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
> here
> is a link
> 
> On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson 
> wrote:
> 
>> Hello all,
>>
>> I have picked up and updated KIP-150. Due to changes to the project since
>> KIP #150 was written there are a few items that need to be updated.
>>
>> First item that changed is the adoption of the Materialized parameter.
>>
>> The second item is the WindowedBy. How the old KIP handles windowing is
>> that it overloads the aggregate function to take in a Window object as well
>> as the other parameters. The current practice to window grouped-streams is
>> to call windowedBy and receive a windowed stream object. The existing
>> interface for a windowed stream made from a grouped stream will not work
>> for cogrouped streams. Hence, we have to make new interfaces for cogrouped
>> windowed streams.
>>
>> Please take a look, I would like to hear your feedback,
>>
>> Walker
>>
> 



signature.asc
Description: OpenPGP digital signature


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

2019-10-14 Thread Walker Carlson
https://cwiki.apache.org/confluence/display/KAFKA/KIP-150+-+Kafka-Streams+Cogroup
here
is a link

On Mon, Oct 14, 2019 at 2:52 PM Walker Carlson 
wrote:

> Hello all,
>
> I have picked up and updated KIP-150. Due to changes to the project since
> KIP #150 was written there are a few items that need to be updated.
>
> First item that changed is the adoption of the Materialized parameter.
>
> The second item is the WindowedBy. How the old KIP handles windowing is
> that it overloads the aggregate function to take in a Window object as well
> as the other parameters. The current practice to window grouped-streams is
> to call windowedBy and receive a windowed stream object. The existing
> interface for a windowed stream made from a grouped stream will not work
> for cogrouped streams. Hence, we have to make new interfaces for cogrouped
> windowed streams.
>
> Please take a look, I would like to hear your feedback,
>
> Walker
>


[DISCUSS] KIP-150 - Kafka-Streams Cogroup

2019-10-14 Thread Walker Carlson
Hello all,

I have picked up and updated KIP-150. Due to changes to the project since
KIP #150 was written there are a few items that need to be updated.

First item that changed is the adoption of the Materialized parameter.

The second item is the WindowedBy. How the old KIP handles windowing is
that it overloads the aggregate function to take in a Window object as well
as the other parameters. The current practice to window grouped-streams is
to call windowedBy and receive a windowed stream object. The existing
interface for a windowed stream made from a grouped stream will not work
for cogrouped streams. Hence, we have to make new interfaces for cogrouped
windowed streams.

Please take a look, I would like to hear your feedback,

Walker


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

2017-06-26 Thread Kyle Winkelman
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"  wrote:

> +1
>
> Thanks,
> Bill
>
> On Wed, Jun 14, 2017 at 8:10 PM, Xavier Léauté 
> wrote:
>
> > +1 from me
> >
> > any stream should be able to initialize the cogroup
> >
> > On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman  >
> > 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"  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 
> > 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:
> > > >
> > > >   CogroupedKStream cogroup(final KGroupedStream
> > > > groupedStream, final Aggregator aggregator)
> > > >
> > > > Note that we do not have a cogroup function that returns
> > > > CogroupedKStream.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck 
> > 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 <
> > matth...@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
> 

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

2017-06-14 Thread Bill Bejeck
+1

Thanks,
Bill

On Wed, Jun 14, 2017 at 8:10 PM, Xavier Léauté  wrote:

> +1 from me
>
> any stream should be able to initialize the cogroup
>
> On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman 
> 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"  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 
> 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:
> > >
> > >   CogroupedKStream cogroup(final KGroupedStream
> > > groupedStream, final Aggregator aggregator)
> > >
> > > Note that we do not have a cogroup function that returns
> > > CogroupedKStream.
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck 
> 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 <
> matth...@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.
> > >> > >>
> > >> > >> 

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

2017-06-14 Thread Xavier Léauté
+1 from me

any stream should be able to initialize the cogroup

On Wed, Jun 14, 2017 at 3:44 PM Kyle Winkelman 
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"  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  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:
> >
> >   CogroupedKStream cogroup(final KGroupedStream
> > groupedStream, final Aggregator aggregator)
> >
> > Note that we do not have a cogroup function that returns
> > CogroupedKStream.
> >
> >
> > Guozhang
> >
> >
> > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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  >
> >> 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.
> >> > >>

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

2017-06-14 Thread Kyle Winkelman
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"  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  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:
>
>   CogroupedKStream cogroup(final KGroupedStream
> groupedStream, final Aggregator aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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 
>> 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.borowie...@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.
>> > 

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

2017-06-14 Thread Kyle Winkelman
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"  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  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:
> >
> >   CogroupedKStream cogroup(final KGroupedStream
> > groupedStream, final Aggregator aggregator)
> >
> > Note that we do not have a cogroup function that returns
> > CogroupedKStream.
> >
> >
> > Guozhang
> >
> >
> > On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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  >
> >> 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.borowie...@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
> >> > >>> 

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

2017-06-14 Thread Guozhang Wang
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  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:
>
>   CogroupedKStream cogroup(final KGroupedStream
> groupedStream, final Aggregator aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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 
>> 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.borowie...@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 grouped1 = builder.stream("topic1").
>> > groupByKey();
>> > >>> KGroupedStream grouped2 = builder.stream("topic2").
>> > groupByKey();
>> > >>> KGroupedStream grouped3 = builder.stream("topic3").
>> > groupByKey();
>> > >>>
>> > >>> KTable coagg = grouped1.aggregate(initializer1, aggregator1,
>> > >>> 

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

2017-06-14 Thread Kyle Winkelman
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"  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:
>
>   CogroupedKStream cogroup(final KGroupedStream
> groupedStream, final Aggregator aggregator)
>
> Note that we do not have a cogroup function that returns
> CogroupedKStream.
>
>
> Guozhang
>
>
> On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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 
> > 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.borowie...@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 grouped1 = builder.stream("topic1").
> > > groupByKey();
> > > >>> KGroupedStream grouped2 = builder.stream("topic2").
> > > groupByKey();
> > > >>> KGroupedStream grouped3 = builder.stream("topic3").
> > > groupByKey();
> > > >>>
> > > >>> KTable 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 

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

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

  CogroupedKStream cogroup(final KGroupedStream
groupedStream, final Aggregator aggregator)

Note that we do not have a cogroup function that returns
CogroupedKStream.


Guozhang


On Tue, Jun 13, 2017 at 2:31 PM, Bill Bejeck  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 
> 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.borowie...@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 grouped1 = builder.stream("topic1").
> > groupByKey();
> > >>> KGroupedStream grouped2 = builder.stream("topic2").
> > groupByKey();
> > >>> KGroupedStream grouped3 = builder.stream("topic3").
> > groupByKey();
> > >>>
> > >>> KTable 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 

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

2017-06-13 Thread Bill Bejeck
+1 on deferring discussion on Serdes until API improvements are ironed out.

On Tue, Jun 13, 2017 at 2:06 PM, Matthias J. Sax 
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.borowie...@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 grouped1 = builder.stream("topic1").
> groupByKey();
> >>> KGroupedStream grouped2 = builder.stream("topic2").
> groupByKey();
> >>> KGroupedStream grouped3 = builder.stream("topic3").
> groupByKey();
> >>>
> >>> KTable 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 initializer,
> >>> Aggregator aggregator, Serde
> aggValueSerde,
> >>> String queryableStoreName)
> >>> The second one only takes a state store supplier, and does not specify
> any
> >>> serde, e.g. KGroupedStream.aggregate(Initializer
> >>> initializer, Aggregator aggregator, final
> >>> StateStoreSupplier storeSupplier)
> >>> Presumably, when specifying a state store supplier it shouldn't be
> >>> necessary to specify an aggregate value serde, since the 

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

2017-06-13 Thread Matthias J. Sax
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" 
>> 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 grouped1 = builder.stream("topic1").groupByKey();
>>> KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
>>> KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
>>>
>>> KTable 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 initializer,
>>> Aggregator aggregator, Serde aggValueSerde,
>>> String queryableStoreName)
>>> The second one only takes a state store supplier, and does not specify any
>>> serde, e.g. KGroupedStream.aggregate(Initializer
>>> initializer, Aggregator aggregator, final
>>> StateStoreSupplier 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
>>> 

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

2017-06-13 Thread Michal Borowiecki

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" 
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 grouped1 = builder.stream("topic1").groupByKey();
KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
KGroupedStream grouped3 = builder.stream("topic3").groupByKey();

KTable 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 initializer,
Aggregator aggregator, Serde aggValueSerde,
String queryableStoreName)
The second one only takes a state store supplier, and does not specify any
serde, e.g. KGroupedStream.aggregate(Initializer
initializer, Aggregator aggregator, final
StateStoreSupplier 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  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.borowie...@openbet.com> wrote:


I feel it would make more sense to move the initializer and serde to the
final aggregate 

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

2017-06-13 Thread Kyle Winkelman
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" 
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 grouped1 = builder.stream("topic1").groupByKey();
> KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
>
> KTable 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 initializer,
> Aggregator aggregator, Serde aggValueSerde,
> String queryableStoreName)
> The second one only takes a state store supplier, and does not specify any
> serde, e.g. KGroupedStream.aggregate(Initializer
> initializer, Aggregator aggregator, final
> StateStoreSupplier 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  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.borowie...@openbet.com> wrote:
>>
>>> I feel it would 

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

2017-06-12 Thread Xavier Léauté
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 initializer,
Aggregator aggregator, Serde aggValueSerde,
String queryableStoreName)
The second one only takes a state store supplier, and does not specify any
serde, e.g. KGroupedStream.aggregate(Initializer
initializer, Aggregator aggregator, final
StateStoreSupplier 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  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.borowie...@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é  
>>  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  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  
>>  wrote:
>>
>>
>> On a second thought... This is the current proposal API
>>
>>
>> ```
>>
>>  CogroupedKStream cogroup(final Initializer initializer,
>>
>> final
>>
>> Aggregator aggregator, final Serde
>> 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 
>>
>> 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  
>> 
>>
>> 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é  
>> 
>> 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, 

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

2017-06-08 Thread Guozhang Wang
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é  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  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  wrote:
>
> > On a second thought... This is the current proposal API
> >
> >
> > ```
> >
> >  CogroupedKStream cogroup(final Initializer initializer,
> final
> > Aggregator aggregator, final Serde
> > 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 
> 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 
> 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é 
> > > 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 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 
> > 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 
> > wrote:
> > >> >
> > >> > > This makes much more sense to me. +1
> > >> > >
> > >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >> winkelman.k...@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" 
> > 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.k...@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.
> > >> > > 

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

2017-06-08 Thread Kyle Winkelman
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é"  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  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  wrote:
>
> > On a second thought... This is the current proposal API
> >
> >
> > ```
> >
> >  CogroupedKStream cogroup(final Initializer initializer,
> final
> > Aggregator aggregator, final Serde
> > 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 
> 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 
> 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é 
> > > 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 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 
> > 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 
> > wrote:
> > >> >
> > >> > > This makes much more sense to me. +1
> > >> > >
> > >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> > >> winkelman.k...@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" 
> > 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.k...@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
> > 

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

2017-06-08 Thread Xavier Léauté
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  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  wrote:

> On a second thought... This is the current proposal API
>
>
> ```
>
>  CogroupedKStream cogroup(final Initializer initializer, final
> Aggregator aggregator, final Serde
> 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  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  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é 
> > 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 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 
> 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 
> wrote:
> >> >
> >> > > This makes much more sense to me. +1
> >> > >
> >> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> >> winkelman.k...@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" 
> 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.k...@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
> >> > > 

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

2017-06-08 Thread Guozhang Wang
On a second thought... This is the current proposal API


```

 CogroupedKStream cogroup(final Initializer initializer, final
Aggregator aggregator, final Serde
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  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  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é 
> 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 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  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  wrote:
>> >
>> > > This makes much more sense to me. +1
>> > >
>> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
>> winkelman.k...@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"  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.k...@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.k...@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" 
>> > wrote:
>> > > 
>> > > > Kyle,
>> > > >
>> > > > Thanks for the explanations, my previous read on the wiki
>> examples
>> > > was
>> > > > wrong.
>> > > >
>> > > > 

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

2017-06-08 Thread Guozhang Wang
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é  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 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  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  wrote:
> >
> > > This makes much more sense to me. +1
> > >
> > > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman <
> winkelman.k...@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"  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.k...@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.k...@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" 
> > 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 

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

2017-06-08 Thread Xavier Léauté
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 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  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  wrote:
>
> > This makes much more sense to me. +1
> >
> > > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman 
> > 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"  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  >
> > >> 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" 
> > >>> 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" 
> 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 

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

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

> This makes much more sense to me. +1
>
> > On Jun 1, 2017, at 10:33 AM, Kyle Winkelman 
> 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"  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 
> >> 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" 
> >>> 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"  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.k...@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" 
> >> wrote:
> 

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

2017-06-02 Thread Jim Jagielski
This makes much more sense to me. +1

> On Jun 1, 2017, at 10:33 AM, Kyle Winkelman  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"  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 
>> 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" 
>>> 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"  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.k...@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" 
>> 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

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

2017-06-01 Thread Kyle Winkelman
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"  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 
> 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" 
> > 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"  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.k...@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" 
> 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 CogroupedKStream#aggregate(Windows);
> > >> >
> > >> >
> > >> > And remove it 

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

2017-05-31 Thread Kyle Winkelman
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"  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"  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 > >
>> 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"  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 CogroupedKStream#aggregate(Windows);
>> >
>> >
>> > And remove it from the cogroup() functions. WDYT?
>> >
>> >
>> > Guozhang
>> >
>> > On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang 
>> 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.k...@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 

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

2017-05-24 Thread Kyle Winkelman
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"  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 
> 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"  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 CogroupedKStream#aggregate(Windows);
> >
> >
> > And remove it from the cogroup() functions. WDYT?
> >
> >
> > Guozhang
> >
> > On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang 
> 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.k...@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é" 
> wrote:
> > >>
> > >> > Sorry to jump on this thread so late. I 

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

2017-05-24 Thread Guozhang Wang
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 
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"  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 CogroupedKStream#aggregate(Windows);
>
>
> And remove it from the cogroup() functions. WDYT?
>
>
> Guozhang
>
> On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang  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.k...@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é"  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 

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

2017-05-24 Thread Kyle Winkelman
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"  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 CogroupedKStream#aggregate(Windows);


And remove it from the cogroup() functions. WDYT?


Guozhang

On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang  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  > 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é"  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 > >
>> > 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 

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

2017-05-23 Thread Guozhang Wang
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 CogroupedKStream#aggregate(Windows);


And remove it from the cogroup() functions. WDYT?


Guozhang

On Tue, May 23, 2017 at 6:22 PM, Guozhang Wang  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  > 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é"  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 > >
>> > 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" 
>> wrote:
>> > >
>> > > > Thanks Kyle, good arguments.
>> > > >
>> > > > Eno
>> > > >
>> > > > > On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>> winkelman.k...@gmail.com
>> > >
>> > > > wrote:
>> > > > >
>> > > > > *- minor: could you add an exact example (similar to what Jay’s

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

2017-05-23 Thread Guozhang Wang
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 
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é"  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 
> > 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"  wrote:
> > >
> > > > Thanks Kyle, good arguments.
> > > >
> > > > Eno
> > > >
> > > > > On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> winkelman.k...@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 

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

2017-05-19 Thread Xavier Léauté
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 
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"  wrote:
>
> > Thanks Kyle, good arguments.
> >
> > Eno
> >
> > > On May 7, 2017, at 5:06 PM, Kyle Winkelman 
> > 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
> > 

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

2017-05-19 Thread Eno Thereska
Kyle, it might make sense to start a VOTE thread, it doesn’t seem there are 
more comments.

Thanks
Eno
> On May 16, 2017, at 6:42 PM, Damian Guy  wrote:
> 
> Yeah - i wish we'd named KGroupedStream GroupedKStream, similarly for
> KGroupedTable.
> 
> On Tue, 16 May 2017 at 17:59 Kyle Winkelman 
> wrote:
> 
>> I have added code blocks and a note about the partial results.
>> 
>> Can I ask why you dont like KCogroupedStream? I just think that because it
>> is created from a KGroupedStream we should keep a similar name format.
>> 
>> On May 16, 2017 9:23 AM, "Damian Guy"  wrote:
>> 
>> Hi Kyle,
>> 
>> Can you put the code examples etc in {code} blocks to make it easier to
>> read?
>> 
>> I think this is probably a pretty common use-case and therefore a
>> worthwhile addition to the API.
>> 
>> I'd suggest dropping the K from KCogroupedStream and calling it
>> CogroupedStream or CogroupedKStream.
>> In your example, wouldn't the output potentially have more partial results?
>> I.e, for each input on any stream you'd potentially see a record produced?
>> I think it is worth mentioning this.
>> 
>> Thanks,
>> Damian
>> 
>> On Tue, 16 May 2017 at 12:26 Kyle Winkelman 
>> wrote:
>> 
>>> No problem, I just wanted to make sure people still had more to say. I
>> will
>>> wait another week.
>>> 
>>> Thanks,
>>> Kyle
>>> 
>>> On May 16, 2017 4:25 AM, "Eno Thereska"  wrote:
>>> 
 Hi Kyle,
 
 Sorry for the delay in reviews, tomorrow is feature freeze deadline (
 
>> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
>>> <
 
>> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
 )
 so people are busier than usual. Stay tuned.
 
 Eno
> On 15 May 2017, at 13:25, Kyle Winkelman 
 wrote:
> 
> Damian Guy, could you let me know if you plan to review this further?
 There
> is no rush, but if you dont have any additional comments I could
>> start
 the
> voting and finish my WIP PR.
> 
> Thanks,
> Kyle
> 
> On May 9, 2017 11:07 AM, "Kyle Winkelman" 
 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" 
>>> wrote:
>> 
>>> Thanks Kyle, good arguments.
>>> 
>>> Eno
>>> 
 On May 7, 2017, at 5:06 PM, Kyle Winkelman <
>>> winkelman.k...@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

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

2017-05-16 Thread Damian Guy
Yeah - i wish we'd named KGroupedStream GroupedKStream, similarly for
KGroupedTable.

On Tue, 16 May 2017 at 17:59 Kyle Winkelman 
wrote:

> I have added code blocks and a note about the partial results.
>
> Can I ask why you dont like KCogroupedStream? I just think that because it
> is created from a KGroupedStream we should keep a similar name format.
>
> On May 16, 2017 9:23 AM, "Damian Guy"  wrote:
>
> Hi Kyle,
>
> Can you put the code examples etc in {code} blocks to make it easier to
> read?
>
> I think this is probably a pretty common use-case and therefore a
> worthwhile addition to the API.
>
> I'd suggest dropping the K from KCogroupedStream and calling it
> CogroupedStream or CogroupedKStream.
> In your example, wouldn't the output potentially have more partial results?
> I.e, for each input on any stream you'd potentially see a record produced?
> I think it is worth mentioning this.
>
> Thanks,
> Damian
>
> On Tue, 16 May 2017 at 12:26 Kyle Winkelman 
> wrote:
>
> > No problem, I just wanted to make sure people still had more to say. I
> will
> > wait another week.
> >
> > Thanks,
> > Kyle
> >
> > On May 16, 2017 4:25 AM, "Eno Thereska"  wrote:
> >
> > > Hi Kyle,
> > >
> > > Sorry for the delay in reviews, tomorrow is feature freeze deadline (
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> > <
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> > >)
> > > so people are busier than usual. Stay tuned.
> > >
> > > Eno
> > > > On 15 May 2017, at 13:25, Kyle Winkelman 
> > > wrote:
> > > >
> > > > Damian Guy, could you let me know if you plan to review this further?
> > > There
> > > > is no rush, but if you dont have any additional comments I could
> start
> > > the
> > > > voting and finish my WIP PR.
> > > >
> > > > Thanks,
> > > > Kyle
> > > >
> > > > On May 9, 2017 11:07 AM, "Kyle Winkelman" 
> > > 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" 
> > wrote:
> > > >>
> > > >>> Thanks Kyle, good arguments.
> > > >>>
> > > >>> Eno
> > > >>>
> > >  On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> > winkelman.k...@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
> > >  

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

2017-05-16 Thread Kyle Winkelman
I have added code blocks and a note about the partial results.

Can I ask why you dont like KCogroupedStream? I just think that because it
is created from a KGroupedStream we should keep a similar name format.

On May 16, 2017 9:23 AM, "Damian Guy"  wrote:

Hi Kyle,

Can you put the code examples etc in {code} blocks to make it easier to
read?

I think this is probably a pretty common use-case and therefore a
worthwhile addition to the API.

I'd suggest dropping the K from KCogroupedStream and calling it
CogroupedStream or CogroupedKStream.
In your example, wouldn't the output potentially have more partial results?
I.e, for each input on any stream you'd potentially see a record produced?
I think it is worth mentioning this.

Thanks,
Damian

On Tue, 16 May 2017 at 12:26 Kyle Winkelman 
wrote:

> No problem, I just wanted to make sure people still had more to say. I
will
> wait another week.
>
> Thanks,
> Kyle
>
> On May 16, 2017 4:25 AM, "Eno Thereska"  wrote:
>
> > Hi Kyle,
> >
> > Sorry for the delay in reviews, tomorrow is feature freeze deadline (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> <
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> >)
> > so people are busier than usual. Stay tuned.
> >
> > Eno
> > > On 15 May 2017, at 13:25, Kyle Winkelman 
> > wrote:
> > >
> > > Damian Guy, could you let me know if you plan to review this further?
> > There
> > > is no rush, but if you dont have any additional comments I could start
> > the
> > > voting and finish my WIP PR.
> > >
> > > Thanks,
> > > Kyle
> > >
> > > On May 9, 2017 11:07 AM, "Kyle Winkelman" 
> > 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" 
> wrote:
> > >>
> > >>> Thanks Kyle, good arguments.
> > >>>
> > >>> Eno
> > >>>
> >  On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> winkelman.k...@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 

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

2017-05-16 Thread Damian Guy
Hi Kyle,

Can you put the code examples etc in {code} blocks to make it easier to
read?

I think this is probably a pretty common use-case and therefore a
worthwhile addition to the API.

I'd suggest dropping the K from KCogroupedStream and calling it
CogroupedStream or CogroupedKStream.
In your example, wouldn't the output potentially have more partial results?
I.e, for each input on any stream you'd potentially see a record produced?
I think it is worth mentioning this.

Thanks,
Damian

On Tue, 16 May 2017 at 12:26 Kyle Winkelman 
wrote:

> No problem, I just wanted to make sure people still had more to say. I will
> wait another week.
>
> Thanks,
> Kyle
>
> On May 16, 2017 4:25 AM, "Eno Thereska"  wrote:
>
> > Hi Kyle,
> >
> > Sorry for the delay in reviews, tomorrow is feature freeze deadline (
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> <
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0
> >)
> > so people are busier than usual. Stay tuned.
> >
> > Eno
> > > On 15 May 2017, at 13:25, Kyle Winkelman 
> > wrote:
> > >
> > > Damian Guy, could you let me know if you plan to review this further?
> > There
> > > is no rush, but if you dont have any additional comments I could start
> > the
> > > voting and finish my WIP PR.
> > >
> > > Thanks,
> > > Kyle
> > >
> > > On May 9, 2017 11:07 AM, "Kyle Winkelman" 
> > 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" 
> wrote:
> > >>
> > >>> Thanks Kyle, good arguments.
> > >>>
> > >>> Eno
> > >>>
> >  On May 7, 2017, at 5:06 PM, Kyle Winkelman <
> winkelman.k...@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.*

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

2017-05-16 Thread Kyle Winkelman
No problem, I just wanted to make sure people still had more to say. I will
wait another week.

Thanks,
Kyle

On May 16, 2017 4:25 AM, "Eno Thereska"  wrote:

> Hi Kyle,
>
> Sorry for the delay in reviews, tomorrow is feature freeze deadline (
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0 <
> https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0>)
> so people are busier than usual. Stay tuned.
>
> Eno
> > On 15 May 2017, at 13:25, Kyle Winkelman 
> wrote:
> >
> > Damian Guy, could you let me know if you plan to review this further?
> There
> > is no rush, but if you dont have any additional comments I could start
> the
> > voting and finish my WIP PR.
> >
> > Thanks,
> > Kyle
> >
> > On May 9, 2017 11:07 AM, "Kyle Winkelman" 
> 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"  wrote:
> >>
> >>> Thanks Kyle, good arguments.
> >>>
> >>> Eno
> >>>
>  On May 7, 2017, at 5:06 PM, Kyle Winkelman 
> >>> 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
> 

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

2017-05-16 Thread Eno Thereska
Hi Kyle,

Sorry for the delay in reviews, tomorrow is feature freeze deadline 
(https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+0.11.0.0 
) so 
people are busier than usual. Stay tuned.

Eno
> On 15 May 2017, at 13:25, Kyle Winkelman  wrote:
> 
> Damian Guy, could you let me know if you plan to review this further? There
> is no rush, but if you dont have any additional comments I could start the
> voting and finish my WIP PR.
> 
> Thanks,
> Kyle
> 
> On May 9, 2017 11:07 AM, "Kyle Winkelman"  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"  wrote:
>> 
>>> Thanks Kyle, good arguments.
>>> 
>>> Eno
>>> 
 On May 7, 2017, at 5:06 PM, Kyle Winkelman 
>>> 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 
>>> 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,
> 

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

2017-05-15 Thread Kyle Winkelman
Damian Guy, could you let me know if you plan to review this further? There
is no rush, but if you dont have any additional comments I could start the
voting and finish my WIP PR.

Thanks,
Kyle

On May 9, 2017 11:07 AM, "Kyle Winkelman"  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"  wrote:
>
>> Thanks Kyle, good arguments.
>>
>> Eno
>>
>> > On May 7, 2017, at 5:06 PM, Kyle Winkelman 
>> 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 
>> 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 

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

2017-05-09 Thread Kyle Winkelman
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"  wrote:

> Thanks Kyle, good arguments.
>
> Eno
>
> > On May 7, 2017, at 5:06 PM, Kyle Winkelman 
> 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 
> 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.
> >>
> >> 

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

2017-05-09 Thread Eno Thereska
Thanks Kyle, good arguments.

Eno

> On May 7, 2017, at 5:06 PM, Kyle Winkelman  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  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  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 

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

2017-05-07 Thread Eno Thereska
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  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 
> 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"  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 
>>> 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 table1 =
 builder.stream("topic1").groupByKey().aggregate(initializer1,
>>> aggregator1,
 aggValueSerde1, storeName1);
 KTable table2 =
 builder.stream("topic2").groupByKey().aggregate(initializer2,
>>> aggregator2,
 aggValueSerde2, storeName2);
 KTable table3 =
 builder.stream("topic3").groupByKey().aggregate(initializer3,
>>> aggregator3,
 aggValueSerde3, storeName3);
 KTable 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 

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

2017-05-05 Thread Jay Kreps
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 
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"  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 
> > 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 table1 =
> > > builder.stream("topic1").groupByKey().aggregate(initializer1,
> > aggregator1,
> > > aggValueSerde1, storeName1);
> > > KTable table2 =
> > > builder.stream("topic2").groupByKey().aggregate(initializer2,
> > aggregator2,
> > > aggValueSerde2, storeName2);
> > > KTable table3 =
> > > builder.stream("topic3").groupByKey().aggregate(initializer3,
> > aggregator3,
> > > aggValueSerde3, storeName3);
> > > KTable 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 grouped1 = builder.stream("topic1").
> groupByKey();
> > > KGroupedStream grouped2 = builder.stream("topic2").
> groupByKey();
> > > KGroupedStream grouped3 = builder.stream("topic3").
> groupByKey();
> > > KTable cogrouped = grouped1.cogroup(initializer1, aggregator1,
> > > aggValueSerde1, storeName1)
> > > .cogroup(grouped2, aggregator2)
> > > .cogroup(grouped3, aggregator3)
> > > .aggregate();
> > >
> > > As you can see this creates 1 

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

2017-05-05 Thread Kyle Winkelman
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"  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 
> 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 table1 =
> > builder.stream("topic1").groupByKey().aggregate(initializer1,
> aggregator1,
> > aggValueSerde1, storeName1);
> > KTable table2 =
> > builder.stream("topic2").groupByKey().aggregate(initializer2,
> aggregator2,
> > aggValueSerde2, storeName2);
> > KTable table3 =
> > builder.stream("topic3").groupByKey().aggregate(initializer3,
> aggregator3,
> > aggValueSerde3, storeName3);
> > KTable 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 grouped1 = builder.stream("topic1").groupByKey();
> > KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> > KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> > KTable 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 
> > 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 

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

2017-05-05 Thread Damian Guy
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  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 table1 =
> builder.stream("topic1").groupByKey().aggregate(initializer1, aggregator1,
> aggValueSerde1, storeName1);
> KTable table2 =
> builder.stream("topic2").groupByKey().aggregate(initializer2, aggregator2,
> aggValueSerde2, storeName2);
> KTable table3 =
> builder.stream("topic3").groupByKey().aggregate(initializer3, aggregator3,
> aggValueSerde3, storeName3);
> KTable 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 grouped1 = builder.stream("topic1").groupByKey();
> KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
> KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
> KTable 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 
> 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 

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

2017-05-04 Thread Kyle Winkelman
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 table1 =
builder.stream("topic1").groupByKey().aggregate(initializer1, aggregator1,
aggValueSerde1, storeName1);
KTable table2 =
builder.stream("topic2").groupByKey().aggregate(initializer2, aggregator2,
aggValueSerde2, storeName2);
KTable table3 =
builder.stream("topic3").groupByKey().aggregate(initializer3, aggregator3,
aggValueSerde3, storeName3);
KTable 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 grouped1 = builder.stream("topic1").groupByKey();
KGroupedStream grouped2 = builder.stream("topic2").groupByKey();
KGroupedStream grouped3 = builder.stream("topic3").groupByKey();
KTable 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 
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" 

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

2017-05-04 Thread Matthias J. Sax
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"  wrote:
> 
>> I’ll look as well asap, sorry, been swamped.
>>
>> Eno
>>> On May 4, 2017, at 6:17 PM, Damian Guy  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 
>> 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" 
>> 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
>

>>
>>
> 



signature.asc
Description: OpenPGP digital signature


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

2017-05-04 Thread Kyle Winkelman
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"  wrote:

> I’ll look as well asap, sorry, been swamped.
>
> Eno
> > On May 4, 2017, at 6:17 PM, Damian Guy  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 
> 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" 
> 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
> >>>
> >>
>
>


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

2017-05-04 Thread Eno Thereska
I’ll look as well asap, sorry, been swamped.

Eno
> On May 4, 2017, at 6:17 PM, Damian Guy  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  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"  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
>>> 
>> 



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

2017-05-04 Thread Damian Guy
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  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"  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
> >
>


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

2017-05-04 Thread Kyle Winkelman
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"  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
>


[DISCUSS] KIP-150 - Kafka-Streams Cogroup

2017-05-01 Thread Kyle Winkelman
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