Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-03-02 Thread John Roesler
Hi, all,

Sorry for the confusion. I didn’t look too closely at it, I was just going by 
the fact that it was listed under the scope of KIP-221.

I agree that the final design of the KIP doesn’t have too much to do with the 
description of KAFKA-4835. Maybe we should remove that ticket from the KIP, and 
also give it a more specific name. 

I’ll ask in the ticket if Levani is also actively working on it, or if he was 
just planning on KIP-221. 

Thanks,
John

On Sun, Mar 1, 2020, at 20:13, Murilo Tavares wrote:
> I agree with Mathias. Can’t see how this KIP/PR helps with the problem
> described in the KAFKA-4835...
> 
> On Sun, Mar 1, 2020 at 2:16 PM Matthias J. Sax  wrote:
> 
> > -BEGIN PGP SIGNED MESSAGE-
> > Hash: SHA512
> >
> > I don't think that KIP-221 addressed the discussed use case.
> >
> > KIP-221 allows to force a repartitioning manually, while the use case
> > describe in the original email was to suppress/skip a repartitioning ste
> > p.
> >
> > The issue to avoid unnecessary repartitioning came up a few time
> > already and I personally believe it's worth to close this gap. But we
> > would need to do a KIP to introduce some API to allow user to tell
> > Kafka Streams that repartitioning is not necessary.
> >
> > In Apache Flink, there is an operator called
> > `reinterpretAsKeyedStream`. We could introduce something similar.
> >
> > - -Matthias
> >
> >
> > On 3/1/20 4:43 AM, John Roesler wrote:
> > > Hi all,
> > >
> > > The KIP is accepted and implemented already, but is blocked on
> > > code review: https://github.com/apache/kafka/pull/7170
> > >
> > > A quick note on the lack of recent progress... It's completely our
> > > fault, the reviews fell by the wayside during the 2.5.0 release
> > > cycle, and we haven't gotten back to it. The contributor, Levani,
> > > has been exceptionally patient with us and continually kept the PR
> > > up-to-date and mergeable since then.
> > >
> > > If you'd like to help get it across the line, Murilo, maybe you can
> > > give it a review?
> > >
> > > Thanks, John
> > >
> > > On Sat, Feb 29, 2020, at 20:52, Guozhang Wang wrote:
> > >> It is in progress, but I was not the main reviewer of that ticket
> > >> so I cannot say for sure. I saw the last update is on Jan/2019 so
> > >> maybe it's a bit loose now.. If you want to pick it up and revive
> > >> the KIP completion feel free to do so :)
> > >>
> > >>
> > >> Guozhang
> > >>
> > >>
> > >> On Fri, Feb 28, 2020 at 5:54 PM Murilo Tavares
> > >>  wrote:
> > >>
> > >>> Guozhang The ticket definitely describes what I’m trying to
> > >>> achieve. And should I be hopeful with the fact it’s in
> > >>> progress? :) Thanks for pointing that out. Murilo
> > >>>
> > >>> On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang
> > >>>  wrote:
> > >>>
> >  Hi Murilo,
> > 
> >  Would this be helping your case?
> >  https://issues.apache.org/jira/browse/KAFKA-4835
> > 
> > 
> >  Guozhang
> > 
> >  On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares
> >   wrote:
> > 
> > > Hi I am currently doing a simple KTable
> > > groupby().aggregate() in
> >  KafkaStreams.
> > > In the groupBy I do need to select a new key, but I know
> > > for sure that
> >  the
> > > new key would still fall in the same partition. Because of
> > > this, I
> >  believe
> > > the repartition would not be necessary, but my question is:
> > > is it
> >  possible
> > > to do a groupBy, changing the key, and tell KafkaStreams to
> > > not create
> >  the
> > > repartition topic? Thanks Murilo
> > >
> > 
> > 
> >  -- -- Guozhang
> > 
> > >>>
> > >>
> > >>
> > >> -- -- Guozhang
> > >>
> > -BEGIN PGP SIGNATURE-
> >
> > iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5cCfQACgkQO4miYXKq
> > /Og0fhAApNlB1LodYwne6x5Fqe5CSveY0c2bmBArDCgmd1BvAstf85ooR9ht05+c
> > 8e1sq/3iLcVaolLDXITK0ptfLB6ZkJRs/sUh4N1ebMNEMtJabAepJ/Y/eEmHJiYX
> > wZ8NcyAZC6QQzFEWavyllMGUVyBMM6ZwFk3/ahwWruCQovWcpxKeItgWqI5thR0B
> > FIVAE6k9qDOfZiu3Qd5Atshfov3PpfG1ezpj4LKqlKfgWhsU+P9U8kfAJVsrgc0i
> > qIPeya1o6hyyAzHnH09EMfNqcRpuJQvYwANq6Br/k+nH4WQQjxXvgE6n8scGJ0TH
> > alAnMmm62UNd88lSltNuF+vf73/omdymJkwMO4sTGK9tC8W5p2OzrIaxfAa8reWU
> > sblSEnH1gHvmIeIzKbb5diqIvwAPNjPMt0FcCJLWUiqjTz1KUHKj/hbAR3AUYxaO
> > PZavruFgQm6jTkuZkWRHW0+5/TytTnR4Ca/KBALQcLcolwMkhYZ5hFIeMW8qWGtR
> > JZHMLEW4doQ66gnWBSaTOSv5LhGOEjp2xQEGoAgO5m8IVfpfwO7Vk6XLa2xjnTN8
> > Z2fUQKIJNxjHgbjOCYZmSnVfpf3egEGmHlbKgaxOOcpnVFee/NOZ5aQxy6MpJfN9
> > 3KvH4yfUNgSEB/b97/W/VdNeJl8dTa11Pd36mMQraUAxcrGcOFA=
> > =DaB8
> > -END PGP SIGNATURE-
> >
>


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-03-01 Thread Murilo Tavares
I agree with Mathias. Can’t see how this KIP/PR helps with the problem
described in the KAFKA-4835...

On Sun, Mar 1, 2020 at 2:16 PM Matthias J. Sax  wrote:

> -BEGIN PGP SIGNED MESSAGE-
> Hash: SHA512
>
> I don't think that KIP-221 addressed the discussed use case.
>
> KIP-221 allows to force a repartitioning manually, while the use case
> describe in the original email was to suppress/skip a repartitioning ste
> p.
>
> The issue to avoid unnecessary repartitioning came up a few time
> already and I personally believe it's worth to close this gap. But we
> would need to do a KIP to introduce some API to allow user to tell
> Kafka Streams that repartitioning is not necessary.
>
> In Apache Flink, there is an operator called
> `reinterpretAsKeyedStream`. We could introduce something similar.
>
> - -Matthias
>
>
> On 3/1/20 4:43 AM, John Roesler wrote:
> > Hi all,
> >
> > The KIP is accepted and implemented already, but is blocked on
> > code review: https://github.com/apache/kafka/pull/7170
> >
> > A quick note on the lack of recent progress... It's completely our
> > fault, the reviews fell by the wayside during the 2.5.0 release
> > cycle, and we haven't gotten back to it. The contributor, Levani,
> > has been exceptionally patient with us and continually kept the PR
> > up-to-date and mergeable since then.
> >
> > If you'd like to help get it across the line, Murilo, maybe you can
> > give it a review?
> >
> > Thanks, John
> >
> > On Sat, Feb 29, 2020, at 20:52, Guozhang Wang wrote:
> >> It is in progress, but I was not the main reviewer of that ticket
> >> so I cannot say for sure. I saw the last update is on Jan/2019 so
> >> maybe it's a bit loose now.. If you want to pick it up and revive
> >> the KIP completion feel free to do so :)
> >>
> >>
> >> Guozhang
> >>
> >>
> >> On Fri, Feb 28, 2020 at 5:54 PM Murilo Tavares
> >>  wrote:
> >>
> >>> Guozhang The ticket definitely describes what I’m trying to
> >>> achieve. And should I be hopeful with the fact it’s in
> >>> progress? :) Thanks for pointing that out. Murilo
> >>>
> >>> On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang
> >>>  wrote:
> >>>
>  Hi Murilo,
> 
>  Would this be helping your case?
>  https://issues.apache.org/jira/browse/KAFKA-4835
> 
> 
>  Guozhang
> 
>  On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares
>   wrote:
> 
> > Hi I am currently doing a simple KTable
> > groupby().aggregate() in
>  KafkaStreams.
> > In the groupBy I do need to select a new key, but I know
> > for sure that
>  the
> > new key would still fall in the same partition. Because of
> > this, I
>  believe
> > the repartition would not be necessary, but my question is:
> > is it
>  possible
> > to do a groupBy, changing the key, and tell KafkaStreams to
> > not create
>  the
> > repartition topic? Thanks Murilo
> >
> 
> 
>  -- -- Guozhang
> 
> >>>
> >>
> >>
> >> -- -- Guozhang
> >>
> -BEGIN PGP SIGNATURE-
>
> iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5cCfQACgkQO4miYXKq
> /Og0fhAApNlB1LodYwne6x5Fqe5CSveY0c2bmBArDCgmd1BvAstf85ooR9ht05+c
> 8e1sq/3iLcVaolLDXITK0ptfLB6ZkJRs/sUh4N1ebMNEMtJabAepJ/Y/eEmHJiYX
> wZ8NcyAZC6QQzFEWavyllMGUVyBMM6ZwFk3/ahwWruCQovWcpxKeItgWqI5thR0B
> FIVAE6k9qDOfZiu3Qd5Atshfov3PpfG1ezpj4LKqlKfgWhsU+P9U8kfAJVsrgc0i
> qIPeya1o6hyyAzHnH09EMfNqcRpuJQvYwANq6Br/k+nH4WQQjxXvgE6n8scGJ0TH
> alAnMmm62UNd88lSltNuF+vf73/omdymJkwMO4sTGK9tC8W5p2OzrIaxfAa8reWU
> sblSEnH1gHvmIeIzKbb5diqIvwAPNjPMt0FcCJLWUiqjTz1KUHKj/hbAR3AUYxaO
> PZavruFgQm6jTkuZkWRHW0+5/TytTnR4Ca/KBALQcLcolwMkhYZ5hFIeMW8qWGtR
> JZHMLEW4doQ66gnWBSaTOSv5LhGOEjp2xQEGoAgO5m8IVfpfwO7Vk6XLa2xjnTN8
> Z2fUQKIJNxjHgbjOCYZmSnVfpf3egEGmHlbKgaxOOcpnVFee/NOZ5aQxy6MpJfN9
> 3KvH4yfUNgSEB/b97/W/VdNeJl8dTa11Pd36mMQraUAxcrGcOFA=
> =DaB8
> -END PGP SIGNATURE-
>


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-03-01 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE-
Hash: SHA512

I don't think that KIP-221 addressed the discussed use case.

KIP-221 allows to force a repartitioning manually, while the use case
describe in the original email was to suppress/skip a repartitioning ste
p.

The issue to avoid unnecessary repartitioning came up a few time
already and I personally believe it's worth to close this gap. But we
would need to do a KIP to introduce some API to allow user to tell
Kafka Streams that repartitioning is not necessary.

In Apache Flink, there is an operator called
`reinterpretAsKeyedStream`. We could introduce something similar.

- -Matthias


On 3/1/20 4:43 AM, John Roesler wrote:
> Hi all,
>
> The KIP is accepted and implemented already, but is blocked on
> code review: https://github.com/apache/kafka/pull/7170
>
> A quick note on the lack of recent progress... It's completely our
> fault, the reviews fell by the wayside during the 2.5.0 release
> cycle, and we haven't gotten back to it. The contributor, Levani,
> has been exceptionally patient with us and continually kept the PR
> up-to-date and mergeable since then.
>
> If you'd like to help get it across the line, Murilo, maybe you can
> give it a review?
>
> Thanks, John
>
> On Sat, Feb 29, 2020, at 20:52, Guozhang Wang wrote:
>> It is in progress, but I was not the main reviewer of that ticket
>> so I cannot say for sure. I saw the last update is on Jan/2019 so
>> maybe it's a bit loose now.. If you want to pick it up and revive
>> the KIP completion feel free to do so :)
>>
>>
>> Guozhang
>>
>>
>> On Fri, Feb 28, 2020 at 5:54 PM Murilo Tavares
>>  wrote:
>>
>>> Guozhang The ticket definitely describes what I’m trying to
>>> achieve. And should I be hopeful with the fact it’s in
>>> progress? :) Thanks for pointing that out. Murilo
>>>
>>> On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang
>>>  wrote:
>>>
 Hi Murilo,

 Would this be helping your case?
 https://issues.apache.org/jira/browse/KAFKA-4835


 Guozhang

 On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares
  wrote:

> Hi I am currently doing a simple KTable
> groupby().aggregate() in
 KafkaStreams.
> In the groupBy I do need to select a new key, but I know
> for sure that
 the
> new key would still fall in the same partition. Because of
> this, I
 believe
> the repartition would not be necessary, but my question is:
> is it
 possible
> to do a groupBy, changing the key, and tell KafkaStreams to
> not create
 the
> repartition topic? Thanks Murilo
>


 -- -- Guozhang

>>>
>>
>>
>> -- -- Guozhang
>>
-BEGIN PGP SIGNATURE-

iQIzBAEBCgAdFiEEI8mthP+5zxXZZdDSO4miYXKq/OgFAl5cCfQACgkQO4miYXKq
/Og0fhAApNlB1LodYwne6x5Fqe5CSveY0c2bmBArDCgmd1BvAstf85ooR9ht05+c
8e1sq/3iLcVaolLDXITK0ptfLB6ZkJRs/sUh4N1ebMNEMtJabAepJ/Y/eEmHJiYX
wZ8NcyAZC6QQzFEWavyllMGUVyBMM6ZwFk3/ahwWruCQovWcpxKeItgWqI5thR0B
FIVAE6k9qDOfZiu3Qd5Atshfov3PpfG1ezpj4LKqlKfgWhsU+P9U8kfAJVsrgc0i
qIPeya1o6hyyAzHnH09EMfNqcRpuJQvYwANq6Br/k+nH4WQQjxXvgE6n8scGJ0TH
alAnMmm62UNd88lSltNuF+vf73/omdymJkwMO4sTGK9tC8W5p2OzrIaxfAa8reWU
sblSEnH1gHvmIeIzKbb5diqIvwAPNjPMt0FcCJLWUiqjTz1KUHKj/hbAR3AUYxaO
PZavruFgQm6jTkuZkWRHW0+5/TytTnR4Ca/KBALQcLcolwMkhYZ5hFIeMW8qWGtR
JZHMLEW4doQ66gnWBSaTOSv5LhGOEjp2xQEGoAgO5m8IVfpfwO7Vk6XLa2xjnTN8
Z2fUQKIJNxjHgbjOCYZmSnVfpf3egEGmHlbKgaxOOcpnVFee/NOZ5aQxy6MpJfN9
3KvH4yfUNgSEB/b97/W/VdNeJl8dTa11Pd36mMQraUAxcrGcOFA=
=DaB8
-END PGP SIGNATURE-


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-02-29 Thread John Roesler
Hi all,

The KIP is accepted and implemented already, but is blocked on code
review: https://github.com/apache/kafka/pull/7170

A quick note on the lack of recent progress... It's completely our fault,
the reviews fell by the wayside during the 2.5.0 release cycle, and we
haven't gotten back to it. The contributor, Levani, has been exceptionally
patient with us and continually kept the PR up-to-date and mergeable
since then.

If you'd like to help get it across the line, Murilo, maybe you can give it
a review?

Thanks,
John

On Sat, Feb 29, 2020, at 20:52, Guozhang Wang wrote:
> It is in progress, but I was not the main reviewer of that ticket so I
> cannot say for sure. I saw the last update is on Jan/2019 so maybe it's a
> bit loose now.. If you want to pick it up and revive the KIP completion
> feel free to do so :)
> 
> 
> Guozhang
> 
> 
> On Fri, Feb 28, 2020 at 5:54 PM Murilo Tavares  wrote:
> 
> > Guozhang
> > The ticket definitely describes what I’m trying to achieve.
> > And should I be hopeful with the fact it’s in progress? :)
> > Thanks for pointing that out.
> > Murilo
> >
> > On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang  wrote:
> >
> > > Hi Murilo,
> > >
> > > Would this be helping your case?
> > > https://issues.apache.org/jira/browse/KAFKA-4835
> > >
> > >
> > > Guozhang
> > >
> > > On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares 
> > > wrote:
> > >
> > > > Hi
> > > > I am currently doing a simple KTable groupby().aggregate() in
> > > KafkaStreams.
> > > > In the groupBy I do need to select a new key, but I know for sure that
> > > the
> > > > new key would still fall in the same partition. Because of this, I
> > > believe
> > > > the repartition would not be necessary, but my question is: is it
> > > possible
> > > > to do a groupBy, changing the key, and tell KafkaStreams to not create
> > > the
> > > > repartition topic?
> > > > Thanks
> > > > Murilo
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> 
> 
> -- 
> -- Guozhang
>


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-02-29 Thread Guozhang Wang
It is in progress, but I was not the main reviewer of that ticket so I
cannot say for sure. I saw the last update is on Jan/2019 so maybe it's a
bit loose now.. If you want to pick it up and revive the KIP completion
feel free to do so :)


Guozhang


On Fri, Feb 28, 2020 at 5:54 PM Murilo Tavares  wrote:

> Guozhang
> The ticket definitely describes what I’m trying to achieve.
> And should I be hopeful with the fact it’s in progress? :)
> Thanks for pointing that out.
> Murilo
>
> On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang  wrote:
>
> > Hi Murilo,
> >
> > Would this be helping your case?
> > https://issues.apache.org/jira/browse/KAFKA-4835
> >
> >
> > Guozhang
> >
> > On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares 
> > wrote:
> >
> > > Hi
> > > I am currently doing a simple KTable groupby().aggregate() in
> > KafkaStreams.
> > > In the groupBy I do need to select a new key, but I know for sure that
> > the
> > > new key would still fall in the same partition. Because of this, I
> > believe
> > > the repartition would not be necessary, but my question is: is it
> > possible
> > > to do a groupBy, changing the key, and tell KafkaStreams to not create
> > the
> > > repartition topic?
> > > Thanks
> > > Murilo
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-- Guozhang


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-02-28 Thread Murilo Tavares
Guozhang
The ticket definitely describes what I’m trying to achieve.
And should I be hopeful with the fact it’s in progress? :)
Thanks for pointing that out.
Murilo

On Fri, Feb 28, 2020 at 2:57 PM Guozhang Wang  wrote:

> Hi Murilo,
>
> Would this be helping your case?
> https://issues.apache.org/jira/browse/KAFKA-4835
>
>
> Guozhang
>
> On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares 
> wrote:
>
> > Hi
> > I am currently doing a simple KTable groupby().aggregate() in
> KafkaStreams.
> > In the groupBy I do need to select a new key, but I know for sure that
> the
> > new key would still fall in the same partition. Because of this, I
> believe
> > the repartition would not be necessary, but my question is: is it
> possible
> > to do a groupBy, changing the key, and tell KafkaStreams to not create
> the
> > repartition topic?
> > Thanks
> > Murilo
> >
>
>
> --
> -- Guozhang
>


Re: KafkaStreams GroupBy with new key. Can I skip repartition?

2020-02-28 Thread Guozhang Wang
Hi Murilo,

Would this be helping your case?
https://issues.apache.org/jira/browse/KAFKA-4835


Guozhang

On Fri, Feb 28, 2020 at 7:01 AM Murilo Tavares  wrote:

> Hi
> I am currently doing a simple KTable groupby().aggregate() in KafkaStreams.
> In the groupBy I do need to select a new key, but I know for sure that the
> new key would still fall in the same partition. Because of this, I believe
> the repartition would not be necessary, but my question is: is it possible
> to do a groupBy, changing the key, and tell KafkaStreams to not create the
> repartition topic?
> Thanks
> Murilo
>


-- 
-- Guozhang