You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Jim Hughes <jh...@confluent.io.INVALID> on 2022/05/10 19:05:44 UTC

[VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Hi all,

I'm asking for a vote on KIP-834:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832

Thanks in advance!

Jim

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Jim Hughes <jh...@confluent.io.INVALID>.
Hi all,

As an update, I wanted to say that I updated my PR to include this
feedback.  Let me know if I should clarify anything on the KIP itself.

Cheers,

Jim

On Thu, Jun 2, 2022 at 12:36 AM Sophie Blee-Goldman
<so...@confluent.io.invalid> wrote:

> Hey Jim, thanks for the update. I'm on the same side as Guozhang here, as
> I've expressed during
> the original discussion I think it would be confusing and possibly harmful
> to continue *any* kind of
> processing or action within Streams while it is "paused". In fact I sort of
> assumed we were including
> active task restoration under the umbrella of standby tasks when we decided
> to pause those as well,
> but since they are technically different I can see why we might want to
> consider them separately.
>
> I would say that for now we should just keep the semantics simple and
> obvious, and if users express
> a desire to pause applications from active processing but allow them to
> catch up as restoring actives,
> lagging standbys, warmup tasks, or so on then we can always add that
> functionality to the feature later on
>
> On Wed, Jun 1, 2022 at 11:41 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Hello Jim,
> >
> > I think If our primary goal would be to reduce resource utilization and
> > potentially to stop the streaming pipeline for investigating possible
> bugs
> > etc, then we should also pause active tasks' restoration as well since
> that
> > 1) may still use resources, and 2) may load in bad data.
> >
> > Guozhang
> >
> >
> >
> >
> >
> >
> >
> > On Wed, Jun 1, 2022 at 5:53 AM Jim Hughes <jh...@confluent.io.invalid>
> > wrote:
> >
> > > Hi all,
> > >
> > > While reviewing my PR for KIP-834, Bruno noticed a case that we may not
> > > have discussed enough.*
> > >
> > > During the discussion, we decided that standby tasks would be paused.
> In
> > > order to do this, there are changes to the StoreChangelogReader around
> > > where it does restorations.  Bruno noticed that the restoration of
> active
> > > tasks is not paused in my PR.
> > >
> > > From my point of view, I was hoping to let active tasks
> > restore/consume/etc
> > > in order that the Kafka Streams instance could transition to RUNNING
> > > (assuming that it was started paused).  I believe Bruno's position is
> > that
> > > if we are pausing restoration for standby tasks, then restoration
> should
> > be
> > > paused for active tasks as well.
> > >
> > > Since this point hasn't been discussed like this, the KIP is unclear
> > about
> > > this detail.
> > >
> > > What do folks think?
> > >
> > > Thanks in advance,
> > >
> > > Jim
> > >
> > > * https://github.com/apache/kafka/pull/12161#discussion_r886732983
> > >
> > > On Mon, May 16, 2022 at 11:07 AM Jim Hughes <jh...@confluent.io>
> > wrote:
> > >
> > > > Hi all,
> > > >
> > > >
> > > > With 5 binding votes (John, Bruno, Sophie, Matthias, Bill) and 4
> > > > non-binding votes (Guozhang, Luke, Leah, Walker), the vote for
> KIP-834
> > > > passes!
> > > >
> > > >
> > > > Thanks all for the great discussion.
> > > >
> > > > I have a PR up here: https://github.com/apache/kafka/pull/12161
> > > >
> > > >
> > > > Thanks in advance for feedback on the PR!
> > > >
> > > >
> > > > Cheers,
> > > >
> > > >
> > > > JIm
> > > >
> > > > On Fri, May 13, 2022 at 12:04 PM Walker Carlson
> > > > <wc...@confluent.io.invalid> wrote:
> > > >
> > > >> +1 from me (non-binding)
> > > >>
> > > >> Walker
> > > >>
> > > >> On Wed, May 11, 2022 at 12:36 PM Leah Thomas
> > > <lthomas@confluent.io.invalid
> > > >> >
> > > >> wrote:
> > > >>
> > > >> > Thanks Jim, great discussion. +1 from me (non-binding)
> > > >> >
> > > >> > Cheers,
> > > >> > Leah
> > > >> >
> > > >> > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com>
> > > wrote:
> > > >> >
> > > >> > > Thanks for the KIP!
> > > >> > >
> > > >> > > +1 (binding)
> > > >> > >
> > > >> > > -Bill
> > > >> > >
> > > >> > > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com>
> > > wrote:
> > > >> > >
> > > >> > > > Hi Jim,
> > > >> > > >
> > > >> > > > I'm +1. (please add some note in KIP about the stream
> resetting
> > > tool
> > > >> > > can't
> > > >> > > > be used in paused state)
> > > >> > > > Thanks for the KIP!
> > > >> > > >
> > > >> > > > Luke
> > > >> > > >
> > > >> > > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > >> > > wrote:
> > > >> > > >
> > > >> > > > > Thanks Jim. +1 from me.
> > > >> > > > >
> > > >> > > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <
> > > mjsax@apache.org
> > > >> >
> > > >> > > > wrote:
> > > >> > > > >
> > > >> > > > > > I had one minor question on the discuss thread. It's
> mainly
> > > >> about
> > > >> > > > > > clarifying and document the user contract. I am fine
> either
> > > way.
> > > >> > > > > >
> > > >> > > > > > +1 (binding)
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > -Matthias
> > > >> > > > > >
> > > >> > > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > >> > > > > > > Thanks for the KIP! +1 (binding)
> > > >> > > > > > >
> > > >> > > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <
> > > >> cadonna@apache.org
> > > >> > >
> > > >> > > > > wrote:
> > > >> > > > > > >
> > > >> > > > > > >> Thanks Jim,
> > > >> > > > > > >>
> > > >> > > > > > >> +1 (binding)
> > > >> > > > > > >>
> > > >> > > > > > >> Best,
> > > >> > > > > > >> Bruno
> > > >> > > > > > >>
> > > >> > > > > > >> On 10.05.22 21:19, John Roesler wrote:
> > > >> > > > > > >>> Thanks Jim,
> > > >> > > > > > >>>
> > > >> > > > > > >>> I’m +1 (binding)
> > > >> > > > > > >>>
> > > >> > > > > > >>> -John
> > > >> > > > > > >>>
> > > >> > > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > > >> > > > > > >>>> Hi all,
> > > >> > > > > > >>>>
> > > >> > > > > > >>>> I'm asking for a vote on KIP-834:
> > > >> > > > > > >>>>
> > > >> > > > > > >>
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > > >> > > > > > >>>>
> > > >> > > > > > >>>> Thanks in advance!
> > > >> > > > > > >>>>
> > > >> > > > > > >>>> Jim
> > > >> > > > > > >>
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > --
> > > >> > > > > -- Guozhang
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Sophie Blee-Goldman <so...@confluent.io.INVALID>.
Hey Jim, thanks for the update. I'm on the same side as Guozhang here, as
I've expressed during
the original discussion I think it would be confusing and possibly harmful
to continue *any* kind of
processing or action within Streams while it is "paused". In fact I sort of
assumed we were including
active task restoration under the umbrella of standby tasks when we decided
to pause those as well,
but since they are technically different I can see why we might want to
consider them separately.

I would say that for now we should just keep the semantics simple and
obvious, and if users express
a desire to pause applications from active processing but allow them to
catch up as restoring actives,
lagging standbys, warmup tasks, or so on then we can always add that
functionality to the feature later on

On Wed, Jun 1, 2022 at 11:41 AM Guozhang Wang <wa...@gmail.com> wrote:

> Hello Jim,
>
> I think If our primary goal would be to reduce resource utilization and
> potentially to stop the streaming pipeline for investigating possible bugs
> etc, then we should also pause active tasks' restoration as well since that
> 1) may still use resources, and 2) may load in bad data.
>
> Guozhang
>
>
>
>
>
>
>
> On Wed, Jun 1, 2022 at 5:53 AM Jim Hughes <jh...@confluent.io.invalid>
> wrote:
>
> > Hi all,
> >
> > While reviewing my PR for KIP-834, Bruno noticed a case that we may not
> > have discussed enough.*
> >
> > During the discussion, we decided that standby tasks would be paused.  In
> > order to do this, there are changes to the StoreChangelogReader around
> > where it does restorations.  Bruno noticed that the restoration of active
> > tasks is not paused in my PR.
> >
> > From my point of view, I was hoping to let active tasks
> restore/consume/etc
> > in order that the Kafka Streams instance could transition to RUNNING
> > (assuming that it was started paused).  I believe Bruno's position is
> that
> > if we are pausing restoration for standby tasks, then restoration should
> be
> > paused for active tasks as well.
> >
> > Since this point hasn't been discussed like this, the KIP is unclear
> about
> > this detail.
> >
> > What do folks think?
> >
> > Thanks in advance,
> >
> > Jim
> >
> > * https://github.com/apache/kafka/pull/12161#discussion_r886732983
> >
> > On Mon, May 16, 2022 at 11:07 AM Jim Hughes <jh...@confluent.io>
> wrote:
> >
> > > Hi all,
> > >
> > >
> > > With 5 binding votes (John, Bruno, Sophie, Matthias, Bill) and 4
> > > non-binding votes (Guozhang, Luke, Leah, Walker), the vote for KIP-834
> > > passes!
> > >
> > >
> > > Thanks all for the great discussion.
> > >
> > > I have a PR up here: https://github.com/apache/kafka/pull/12161
> > >
> > >
> > > Thanks in advance for feedback on the PR!
> > >
> > >
> > > Cheers,
> > >
> > >
> > > JIm
> > >
> > > On Fri, May 13, 2022 at 12:04 PM Walker Carlson
> > > <wc...@confluent.io.invalid> wrote:
> > >
> > >> +1 from me (non-binding)
> > >>
> > >> Walker
> > >>
> > >> On Wed, May 11, 2022 at 12:36 PM Leah Thomas
> > <lthomas@confluent.io.invalid
> > >> >
> > >> wrote:
> > >>
> > >> > Thanks Jim, great discussion. +1 from me (non-binding)
> > >> >
> > >> > Cheers,
> > >> > Leah
> > >> >
> > >> > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com>
> > wrote:
> > >> >
> > >> > > Thanks for the KIP!
> > >> > >
> > >> > > +1 (binding)
> > >> > >
> > >> > > -Bill
> > >> > >
> > >> > > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com>
> > wrote:
> > >> > >
> > >> > > > Hi Jim,
> > >> > > >
> > >> > > > I'm +1. (please add some note in KIP about the stream resetting
> > tool
> > >> > > can't
> > >> > > > be used in paused state)
> > >> > > > Thanks for the KIP!
> > >> > > >
> > >> > > > Luke
> > >> > > >
> > >> > > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <
> wangguoz@gmail.com
> > >
> > >> > > wrote:
> > >> > > >
> > >> > > > > Thanks Jim. +1 from me.
> > >> > > > >
> > >> > > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <
> > mjsax@apache.org
> > >> >
> > >> > > > wrote:
> > >> > > > >
> > >> > > > > > I had one minor question on the discuss thread. It's mainly
> > >> about
> > >> > > > > > clarifying and document the user contract. I am fine either
> > way.
> > >> > > > > >
> > >> > > > > > +1 (binding)
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > -Matthias
> > >> > > > > >
> > >> > > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > >> > > > > > > Thanks for the KIP! +1 (binding)
> > >> > > > > > >
> > >> > > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <
> > >> cadonna@apache.org
> > >> > >
> > >> > > > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Thanks Jim,
> > >> > > > > > >>
> > >> > > > > > >> +1 (binding)
> > >> > > > > > >>
> > >> > > > > > >> Best,
> > >> > > > > > >> Bruno
> > >> > > > > > >>
> > >> > > > > > >> On 10.05.22 21:19, John Roesler wrote:
> > >> > > > > > >>> Thanks Jim,
> > >> > > > > > >>>
> > >> > > > > > >>> I’m +1 (binding)
> > >> > > > > > >>>
> > >> > > > > > >>> -John
> > >> > > > > > >>>
> > >> > > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > >> > > > > > >>>> Hi all,
> > >> > > > > > >>>>
> > >> > > > > > >>>> I'm asking for a vote on KIP-834:
> > >> > > > > > >>>>
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > >> > > > > > >>>>
> > >> > > > > > >>>> Thanks in advance!
> > >> > > > > > >>>>
> > >> > > > > > >>>> Jim
> > >> > > > > > >>
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > > >
> > >> > > > > --
> > >> > > > > -- Guozhang
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

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

I think If our primary goal would be to reduce resource utilization and
potentially to stop the streaming pipeline for investigating possible bugs
etc, then we should also pause active tasks' restoration as well since that
1) may still use resources, and 2) may load in bad data.

Guozhang







On Wed, Jun 1, 2022 at 5:53 AM Jim Hughes <jh...@confluent.io.invalid>
wrote:

> Hi all,
>
> While reviewing my PR for KIP-834, Bruno noticed a case that we may not
> have discussed enough.*
>
> During the discussion, we decided that standby tasks would be paused.  In
> order to do this, there are changes to the StoreChangelogReader around
> where it does restorations.  Bruno noticed that the restoration of active
> tasks is not paused in my PR.
>
> From my point of view, I was hoping to let active tasks restore/consume/etc
> in order that the Kafka Streams instance could transition to RUNNING
> (assuming that it was started paused).  I believe Bruno's position is that
> if we are pausing restoration for standby tasks, then restoration should be
> paused for active tasks as well.
>
> Since this point hasn't been discussed like this, the KIP is unclear about
> this detail.
>
> What do folks think?
>
> Thanks in advance,
>
> Jim
>
> * https://github.com/apache/kafka/pull/12161#discussion_r886732983
>
> On Mon, May 16, 2022 at 11:07 AM Jim Hughes <jh...@confluent.io> wrote:
>
> > Hi all,
> >
> >
> > With 5 binding votes (John, Bruno, Sophie, Matthias, Bill) and 4
> > non-binding votes (Guozhang, Luke, Leah, Walker), the vote for KIP-834
> > passes!
> >
> >
> > Thanks all for the great discussion.
> >
> > I have a PR up here: https://github.com/apache/kafka/pull/12161
> >
> >
> > Thanks in advance for feedback on the PR!
> >
> >
> > Cheers,
> >
> >
> > JIm
> >
> > On Fri, May 13, 2022 at 12:04 PM Walker Carlson
> > <wc...@confluent.io.invalid> wrote:
> >
> >> +1 from me (non-binding)
> >>
> >> Walker
> >>
> >> On Wed, May 11, 2022 at 12:36 PM Leah Thomas
> <lthomas@confluent.io.invalid
> >> >
> >> wrote:
> >>
> >> > Thanks Jim, great discussion. +1 from me (non-binding)
> >> >
> >> > Cheers,
> >> > Leah
> >> >
> >> > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com>
> wrote:
> >> >
> >> > > Thanks for the KIP!
> >> > >
> >> > > +1 (binding)
> >> > >
> >> > > -Bill
> >> > >
> >> > > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com>
> wrote:
> >> > >
> >> > > > Hi Jim,
> >> > > >
> >> > > > I'm +1. (please add some note in KIP about the stream resetting
> tool
> >> > > can't
> >> > > > be used in paused state)
> >> > > > Thanks for the KIP!
> >> > > >
> >> > > > Luke
> >> > > >
> >> > > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wangguoz@gmail.com
> >
> >> > > wrote:
> >> > > >
> >> > > > > Thanks Jim. +1 from me.
> >> > > > >
> >> > > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <
> mjsax@apache.org
> >> >
> >> > > > wrote:
> >> > > > >
> >> > > > > > I had one minor question on the discuss thread. It's mainly
> >> about
> >> > > > > > clarifying and document the user contract. I am fine either
> way.
> >> > > > > >
> >> > > > > > +1 (binding)
> >> > > > > >
> >> > > > > >
> >> > > > > > -Matthias
> >> > > > > >
> >> > > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> >> > > > > > > Thanks for the KIP! +1 (binding)
> >> > > > > > >
> >> > > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <
> >> cadonna@apache.org
> >> > >
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > >> Thanks Jim,
> >> > > > > > >>
> >> > > > > > >> +1 (binding)
> >> > > > > > >>
> >> > > > > > >> Best,
> >> > > > > > >> Bruno
> >> > > > > > >>
> >> > > > > > >> On 10.05.22 21:19, John Roesler wrote:
> >> > > > > > >>> Thanks Jim,
> >> > > > > > >>>
> >> > > > > > >>> I’m +1 (binding)
> >> > > > > > >>>
> >> > > > > > >>> -John
> >> > > > > > >>>
> >> > > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> >> > > > > > >>>> Hi all,
> >> > > > > > >>>>
> >> > > > > > >>>> I'm asking for a vote on KIP-834:
> >> > > > > > >>>>
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> >> > > > > > >>>>
> >> > > > > > >>>> Thanks in advance!
> >> > > > > > >>>>
> >> > > > > > >>>> Jim
> >> > > > > > >>
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > > >
> >> > > > > --
> >> > > > > -- Guozhang
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >
>


-- 
-- Guozhang

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Jim Hughes <jh...@confluent.io.INVALID>.
Hi all,

While reviewing my PR for KIP-834, Bruno noticed a case that we may not
have discussed enough.*

During the discussion, we decided that standby tasks would be paused.  In
order to do this, there are changes to the StoreChangelogReader around
where it does restorations.  Bruno noticed that the restoration of active
tasks is not paused in my PR.

From my point of view, I was hoping to let active tasks restore/consume/etc
in order that the Kafka Streams instance could transition to RUNNING
(assuming that it was started paused).  I believe Bruno's position is that
if we are pausing restoration for standby tasks, then restoration should be
paused for active tasks as well.

Since this point hasn't been discussed like this, the KIP is unclear about
this detail.

What do folks think?

Thanks in advance,

Jim

* https://github.com/apache/kafka/pull/12161#discussion_r886732983

On Mon, May 16, 2022 at 11:07 AM Jim Hughes <jh...@confluent.io> wrote:

> Hi all,
>
>
> With 5 binding votes (John, Bruno, Sophie, Matthias, Bill) and 4
> non-binding votes (Guozhang, Luke, Leah, Walker), the vote for KIP-834
> passes!
>
>
> Thanks all for the great discussion.
>
> I have a PR up here: https://github.com/apache/kafka/pull/12161
>
>
> Thanks in advance for feedback on the PR!
>
>
> Cheers,
>
>
> JIm
>
> On Fri, May 13, 2022 at 12:04 PM Walker Carlson
> <wc...@confluent.io.invalid> wrote:
>
>> +1 from me (non-binding)
>>
>> Walker
>>
>> On Wed, May 11, 2022 at 12:36 PM Leah Thomas <lthomas@confluent.io.invalid
>> >
>> wrote:
>>
>> > Thanks Jim, great discussion. +1 from me (non-binding)
>> >
>> > Cheers,
>> > Leah
>> >
>> > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com> wrote:
>> >
>> > > Thanks for the KIP!
>> > >
>> > > +1 (binding)
>> > >
>> > > -Bill
>> > >
>> > > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com> wrote:
>> > >
>> > > > Hi Jim,
>> > > >
>> > > > I'm +1. (please add some note in KIP about the stream resetting tool
>> > > can't
>> > > > be used in paused state)
>> > > > Thanks for the KIP!
>> > > >
>> > > > Luke
>> > > >
>> > > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com>
>> > > wrote:
>> > > >
>> > > > > Thanks Jim. +1 from me.
>> > > > >
>> > > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mjsax@apache.org
>> >
>> > > > wrote:
>> > > > >
>> > > > > > I had one minor question on the discuss thread. It's mainly
>> about
>> > > > > > clarifying and document the user contract. I am fine either way.
>> > > > > >
>> > > > > > +1 (binding)
>> > > > > >
>> > > > > >
>> > > > > > -Matthias
>> > > > > >
>> > > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
>> > > > > > > Thanks for the KIP! +1 (binding)
>> > > > > > >
>> > > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <
>> cadonna@apache.org
>> > >
>> > > > > wrote:
>> > > > > > >
>> > > > > > >> Thanks Jim,
>> > > > > > >>
>> > > > > > >> +1 (binding)
>> > > > > > >>
>> > > > > > >> Best,
>> > > > > > >> Bruno
>> > > > > > >>
>> > > > > > >> On 10.05.22 21:19, John Roesler wrote:
>> > > > > > >>> Thanks Jim,
>> > > > > > >>>
>> > > > > > >>> I’m +1 (binding)
>> > > > > > >>>
>> > > > > > >>> -John
>> > > > > > >>>
>> > > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
>> > > > > > >>>> Hi all,
>> > > > > > >>>>
>> > > > > > >>>> I'm asking for a vote on KIP-834:
>> > > > > > >>>>
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
>> > > > > > >>>>
>> > > > > > >>>> Thanks in advance!
>> > > > > > >>>>
>> > > > > > >>>> Jim
>> > > > > > >>
>> > > > > > >
>> > > > > >
>> > > > >
>> > > > >
>> > > > > --
>> > > > > -- Guozhang
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Jim Hughes <jh...@confluent.io.INVALID>.
Hi all,


With 5 binding votes (John, Bruno, Sophie, Matthias, Bill) and 4
non-binding votes (Guozhang, Luke, Leah, Walker), the vote for KIP-834
passes!


Thanks all for the great discussion.

I have a PR up here: https://github.com/apache/kafka/pull/12161


Thanks in advance for feedback on the PR!


Cheers,


JIm

On Fri, May 13, 2022 at 12:04 PM Walker Carlson
<wc...@confluent.io.invalid> wrote:

> +1 from me (non-binding)
>
> Walker
>
> On Wed, May 11, 2022 at 12:36 PM Leah Thomas <lthomas@confluent.io.invalid
> >
> wrote:
>
> > Thanks Jim, great discussion. +1 from me (non-binding)
> >
> > Cheers,
> > Leah
> >
> > On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com> wrote:
> >
> > > Thanks for the KIP!
> > >
> > > +1 (binding)
> > >
> > > -Bill
> > >
> > > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com> wrote:
> > >
> > > > Hi Jim,
> > > >
> > > > I'm +1. (please add some note in KIP about the stream resetting tool
> > > can't
> > > > be used in paused state)
> > > > Thanks for the KIP!
> > > >
> > > > Luke
> > > >
> > > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Thanks Jim. +1 from me.
> > > > >
> > > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org>
> > > > wrote:
> > > > >
> > > > > > I had one minor question on the discuss thread. It's mainly about
> > > > > > clarifying and document the user contract. I am fine either way.
> > > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > > > > > Thanks for the KIP! +1 (binding)
> > > > > > >
> > > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <
> cadonna@apache.org
> > >
> > > > > wrote:
> > > > > > >
> > > > > > >> Thanks Jim,
> > > > > > >>
> > > > > > >> +1 (binding)
> > > > > > >>
> > > > > > >> Best,
> > > > > > >> Bruno
> > > > > > >>
> > > > > > >> On 10.05.22 21:19, John Roesler wrote:
> > > > > > >>> Thanks Jim,
> > > > > > >>>
> > > > > > >>> I’m +1 (binding)
> > > > > > >>>
> > > > > > >>> -John
> > > > > > >>>
> > > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > > > > > >>>> Hi all,
> > > > > > >>>>
> > > > > > >>>> I'm asking for a vote on KIP-834:
> > > > > > >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > > > > > >>>>
> > > > > > >>>> Thanks in advance!
> > > > > > >>>>
> > > > > > >>>> Jim
> > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Walker Carlson <wc...@confluent.io.INVALID>.
+1 from me (non-binding)

Walker

On Wed, May 11, 2022 at 12:36 PM Leah Thomas <lt...@confluent.io.invalid>
wrote:

> Thanks Jim, great discussion. +1 from me (non-binding)
>
> Cheers,
> Leah
>
> On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com> wrote:
>
> > Thanks for the KIP!
> >
> > +1 (binding)
> >
> > -Bill
> >
> > On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com> wrote:
> >
> > > Hi Jim,
> > >
> > > I'm +1. (please add some note in KIP about the stream resetting tool
> > can't
> > > be used in paused state)
> > > Thanks for the KIP!
> > >
> > > Luke
> > >
> > > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Thanks Jim. +1 from me.
> > > >
> > > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org>
> > > wrote:
> > > >
> > > > > I had one minor question on the discuss thread. It's mainly about
> > > > > clarifying and document the user contract. I am fine either way.
> > > > >
> > > > > +1 (binding)
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > > > > Thanks for the KIP! +1 (binding)
> > > > > >
> > > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <cadonna@apache.org
> >
> > > > wrote:
> > > > > >
> > > > > >> Thanks Jim,
> > > > > >>
> > > > > >> +1 (binding)
> > > > > >>
> > > > > >> Best,
> > > > > >> Bruno
> > > > > >>
> > > > > >> On 10.05.22 21:19, John Roesler wrote:
> > > > > >>> Thanks Jim,
> > > > > >>>
> > > > > >>> I’m +1 (binding)
> > > > > >>>
> > > > > >>> -John
> > > > > >>>
> > > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > > > > >>>> Hi all,
> > > > > >>>>
> > > > > >>>> I'm asking for a vote on KIP-834:
> > > > > >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > > > > >>>>
> > > > > >>>> Thanks in advance!
> > > > > >>>>
> > > > > >>>> Jim
> > > > > >>
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Leah Thomas <lt...@confluent.io.INVALID>.
Thanks Jim, great discussion. +1 from me (non-binding)

Cheers,
Leah

On Wed, May 11, 2022 at 10:14 AM Bill Bejeck <bb...@gmail.com> wrote:

> Thanks for the KIP!
>
> +1 (binding)
>
> -Bill
>
> On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com> wrote:
>
> > Hi Jim,
> >
> > I'm +1. (please add some note in KIP about the stream resetting tool
> can't
> > be used in paused state)
> > Thanks for the KIP!
> >
> > Luke
> >
> > On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Thanks Jim. +1 from me.
> > >
> > > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org>
> > wrote:
> > >
> > > > I had one minor question on the discuss thread. It's mainly about
> > > > clarifying and document the user contract. I am fine either way.
> > > >
> > > > +1 (binding)
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > > > Thanks for the KIP! +1 (binding)
> > > > >
> > > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org>
> > > wrote:
> > > > >
> > > > >> Thanks Jim,
> > > > >>
> > > > >> +1 (binding)
> > > > >>
> > > > >> Best,
> > > > >> Bruno
> > > > >>
> > > > >> On 10.05.22 21:19, John Roesler wrote:
> > > > >>> Thanks Jim,
> > > > >>>
> > > > >>> I’m +1 (binding)
> > > > >>>
> > > > >>> -John
> > > > >>>
> > > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > > > >>>> Hi all,
> > > > >>>>
> > > > >>>> I'm asking for a vote on KIP-834:
> > > > >>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > > > >>>>
> > > > >>>> Thanks in advance!
> > > > >>>>
> > > > >>>> Jim
> > > > >>
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Bill Bejeck <bb...@gmail.com>.
Thanks for the KIP!

+1 (binding)

-Bill

On Wed, May 11, 2022 at 9:36 AM Luke Chen <sh...@gmail.com> wrote:

> Hi Jim,
>
> I'm +1. (please add some note in KIP about the stream resetting tool can't
> be used in paused state)
> Thanks for the KIP!
>
> Luke
>
> On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com> wrote:
>
> > Thanks Jim. +1 from me.
> >
> > On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> > > I had one minor question on the discuss thread. It's mainly about
> > > clarifying and document the user contract. I am fine either way.
> > >
> > > +1 (binding)
> > >
> > >
> > > -Matthias
> > >
> > > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > > Thanks for the KIP! +1 (binding)
> > > >
> > > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org>
> > wrote:
> > > >
> > > >> Thanks Jim,
> > > >>
> > > >> +1 (binding)
> > > >>
> > > >> Best,
> > > >> Bruno
> > > >>
> > > >> On 10.05.22 21:19, John Roesler wrote:
> > > >>> Thanks Jim,
> > > >>>
> > > >>> I’m +1 (binding)
> > > >>>
> > > >>> -John
> > > >>>
> > > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > > >>>> Hi all,
> > > >>>>
> > > >>>> I'm asking for a vote on KIP-834:
> > > >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > > >>>>
> > > >>>> Thanks in advance!
> > > >>>>
> > > >>>> Jim
> > > >>
> > > >
> > >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Luke Chen <sh...@gmail.com>.
Hi Jim,

I'm +1. (please add some note in KIP about the stream resetting tool can't
be used in paused state)
Thanks for the KIP!

Luke

On Wed, May 11, 2022 at 9:09 AM Guozhang Wang <wa...@gmail.com> wrote:

> Thanks Jim. +1 from me.
>
> On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org> wrote:
>
> > I had one minor question on the discuss thread. It's mainly about
> > clarifying and document the user contract. I am fine either way.
> >
> > +1 (binding)
> >
> >
> > -Matthias
> >
> > On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > > Thanks for the KIP! +1 (binding)
> > >
> > > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org>
> wrote:
> > >
> > >> Thanks Jim,
> > >>
> > >> +1 (binding)
> > >>
> > >> Best,
> > >> Bruno
> > >>
> > >> On 10.05.22 21:19, John Roesler wrote:
> > >>> Thanks Jim,
> > >>>
> > >>> I’m +1 (binding)
> > >>>
> > >>> -John
> > >>>
> > >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> > >>>> Hi all,
> > >>>>
> > >>>> I'm asking for a vote on KIP-834:
> > >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> > >>>>
> > >>>> Thanks in advance!
> > >>>>
> > >>>> Jim
> > >>
> > >
> >
>
>
> --
> -- Guozhang
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Guozhang Wang <wa...@gmail.com>.
Thanks Jim. +1 from me.

On Tue, May 10, 2022 at 4:51 PM Matthias J. Sax <mj...@apache.org> wrote:

> I had one minor question on the discuss thread. It's mainly about
> clarifying and document the user contract. I am fine either way.
>
> +1 (binding)
>
>
> -Matthias
>
> On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> > Thanks for the KIP! +1 (binding)
> >
> > On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org> wrote:
> >
> >> Thanks Jim,
> >>
> >> +1 (binding)
> >>
> >> Best,
> >> Bruno
> >>
> >> On 10.05.22 21:19, John Roesler wrote:
> >>> Thanks Jim,
> >>>
> >>> I’m +1 (binding)
> >>>
> >>> -John
> >>>
> >>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> >>>> Hi all,
> >>>>
> >>>> I'm asking for a vote on KIP-834:
> >>>>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> >>>>
> >>>> Thanks in advance!
> >>>>
> >>>> Jim
> >>
> >
>


-- 
-- Guozhang

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by "Matthias J. Sax" <mj...@apache.org>.
I had one minor question on the discuss thread. It's mainly about 
clarifying and document the user contract. I am fine either way.

+1 (binding)


-Matthias

On 5/10/22 12:32 PM, Sophie Blee-Goldman wrote:
> Thanks for the KIP! +1 (binding)
> 
> On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org> wrote:
> 
>> Thanks Jim,
>>
>> +1 (binding)
>>
>> Best,
>> Bruno
>>
>> On 10.05.22 21:19, John Roesler wrote:
>>> Thanks Jim,
>>>
>>> I’m +1 (binding)
>>>
>>> -John
>>>
>>> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
>>>> Hi all,
>>>>
>>>> I'm asking for a vote on KIP-834:
>>>>
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
>>>>
>>>> Thanks in advance!
>>>>
>>>> Jim
>>
> 

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Sophie Blee-Goldman <so...@confluent.io.INVALID>.
Thanks for the KIP! +1 (binding)

On Tue, May 10, 2022, 12:24 PM Bruno Cadonna <ca...@apache.org> wrote:

> Thanks Jim,
>
> +1 (binding)
>
> Best,
> Bruno
>
> On 10.05.22 21:19, John Roesler wrote:
> > Thanks Jim,
> >
> > I’m +1 (binding)
> >
> > -John
> >
> > On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> >> Hi all,
> >>
> >> I'm asking for a vote on KIP-834:
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
> >>
> >> Thanks in advance!
> >>
> >> Jim
>

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by Bruno Cadonna <ca...@apache.org>.
Thanks Jim,

+1 (binding)

Best,
Bruno

On 10.05.22 21:19, John Roesler wrote:
> Thanks Jim,
> 
> I’m +1 (binding)
> 
> -John
> 
> On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
>> Hi all,
>>
>> I'm asking for a vote on KIP-834:
>> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
>>
>> Thanks in advance!
>>
>> Jim

Re: [VOTE] KIP-834: Pause / Resume KafkaStreams Topologies

Posted by John Roesler <vv...@apache.org>.
Thanks Jim,

I’m +1 (binding)

-John

On Tue, May 10, 2022, at 14:05, Jim Hughes wrote:
> Hi all,
>
> I'm asking for a vote on KIP-834:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
>
> Thanks in advance!
>
> Jim