Kyle,

thanks a lot for the KIP. Maybe I am a little slow, but I could not
follow completely. Could you maybe add a more concrete example, like 3
streams with 3 records each (plus expected result), and show the
difference between current way to to implement it and the proposed API?
This could also cover the internal processing to see what store calls
would be required for both approaches etc.

I think, it's pretty advanced stuff you propose, and it would help to
understand it better.

Thanks a lot!


-Matthias



On 5/4/17 11:39 AM, Kyle Winkelman wrote:
> I have made a pull request. It can be found here.
> 
> https://github.com/apache/kafka/pull/2975
> 
> I plan to write some more unit tests for my classes and get around to
> writing documentation for the public api additions.
> 
> One thing I was curious about is during the KCogroupedStreamImpl#aggregate
> method I pass null to the KGroupedStream#repartitionIfRequired method. I
> can't supply the store name because if more than one grouped stream
> repartitions an error is thrown. Is there some name that someone can
> recommend or should I leave the null and allow it to fall back to the
> KGroupedStream.name?
> 
> Should this be expanded to handle grouped tables? This would be pretty easy
> for a normal aggregate but one allowing session stores and windowed stores
> would required KTableSessionWindowAggregate and KTableWindowAggregate
> implementations.
> 
> Thanks,
> Kyle
> 
> On May 4, 2017 1:24 PM, "Eno Thereska" <eno.there...@gmail.com> wrote:
> 
>> I’ll look as well asap, sorry, been swamped.
>>
>> Eno
>>> On May 4, 2017, at 6:17 PM, Damian Guy <damian....@gmail.com> wrote:
>>>
>>> Hi Kyle,
>>>
>>> Thanks for the KIP. I apologize that i haven't had the chance to look at
>>> the KIP yet, but will schedule some time to look into it tomorrow. For
>> the
>>> implementation, can you raise a PR against kafka trunk and mark it as
>> WIP?
>>> It will be easier to review what you have done.
>>>
>>> Thanks,
>>> Damian
>>>
>>> On Thu, 4 May 2017 at 11:50 Kyle Winkelman <winkelman.k...@gmail.com>
>> wrote:
>>>
>>>> I am replying to this in hopes it will draw some attention to my KIP as
>> I
>>>> haven't heard from anyone in a couple days. This is my first KIP and my
>>>> first large contribution to the project so I'm sure I did something
>> wrong.
>>>> ;)
>>>>
>>>> On May 1, 2017 4:18 PM, "Kyle Winkelman" <winkelman.k...@gmail.com>
>> wrote:
>>>>
>>>>> Hello all,
>>>>>
>>>>> I have created KIP-150 to facilitate discussion about adding cogroup to
>>>>> the streams DSL.
>>>>>
>>>>> Please find the KIP here:
>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
>>>>> 150+-+Kafka-Streams+Cogroup
>>>>>
>>>>> Please find my initial implementation here:
>>>>> https://github.com/KyleWinkelman/kafka
>>>>>
>>>>> Thanks,
>>>>> Kyle Winkelman
>>>>>
>>>>
>>
>>
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to