Hi Luke, John,
Thanks for bringing up this and also sorting it out!
I have added a note to the KIP.
Thanks,
Jim
On Wed, May 11, 2022 at 9:34 AM Luke Chen wrote:
> Thanks John!
> It makes sense.
> I have no other questions as long as it is documented in the KIP.
>
> Thank you.
> Luke
>
> On
Thanks John!
It makes sense.
I have no other questions as long as it is documented in the KIP.
Thank you.
Luke
On Wed, May 11, 2022 at 9:15 PM John Roesler wrote:
> Hi Luke,
>
> It’s not my KIP, but my two cents is that users should not run the reset
> tool while the application is paused.
>
>
Hi Luke,
It’s not my KIP, but my two cents is that users should not run the reset tool
while the application is paused.
The reset tool should only be run while the whole app is shut down because it
messes with a lot of internal state bits without synchronization. Leaving the
app running (even
Hi Jim,
Thanks for the KIP. Overall LGTM!
One late question:
Could we run the stream resetter tool (i.e.
kafka-streams-application-reset.sh) during pause state?
I can imagine there's a use case that after pausing for a while, user just
want to continue with the latest offset, and skipping the
Hi Matthias,
I like it. I've updated the KIP to reflect that detail; I put the details
in the docs for pause.
Cheers,
Jim
On Tue, May 10, 2022 at 7:51 PM Matthias J. Sax wrote:
> Thanks for the KIP. Overall LGTM.
>
> Can we clarify one question: would it be allowed to call `pause()`
>
Thanks for the KIP. Overall LGTM.
Can we clarify one question: would it be allowed to call `pause()`
before calling `start()`? I don't see any reason why we would need to
disallow it?
It could be helpful to start a KafkaStreams client in paused state --
otherwise there is a race between
Hi Bill, all,
Thank you. I've updated the KIP to reflect pausing standby tasks as well.
I think all the outstanding points have been addressed and I'm going to
start the vote thread!
Cheers,
Jim
On Tue, May 10, 2022 at 2:43 PM Bill Bejeck wrote:
> Hi Jim,
>
> After reading the comments on
Hi Jim,
After reading the comments on the KIP, I agree that it makes sense to pause
all activities and any changes can be made later on.
Thanks,
Bill
On Tue, May 10, 2022 at 4:03 AM Bruno Cadonna wrote:
> Hi Jim,
>
> Thanks for the KIP!
>
> I am fine with the KIP in general.
>
> However, I am
Hi Jim,
Thanks for the KIP!
I am fine with the KIP in general.
However, I am with Sophie and John to also pause the standbys for the
reasons they brought up. Is there a specific reason you want to keep
standbys going? It feels like premature optimization to me. We can still
add keeping
Thanks Jim, just one note/question on the standby tasks:
At the minute, my moderately held position is that standby tasks ought to
> continue reading and remain caught up. If standby tasks would run out of
> space, there are probably bigger problems.
For a single node application, or when the
Thanks for the updates, Jim!
After this discussion and your updates, this KIP looks good to me.
Thanks,
John
On Mon, May 9, 2022, at 17:52, Jim Hughes wrote:
> Hi Sophie, all,
>
> I've updated the KIP with feedback from the discussion so far:
>
Hi Sophie, all,
I've updated the KIP with feedback from the discussion so far:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
As a terse summary of my current position:
Pausing will only stop processing and punctuation (respecting modular
topologies).
Paused
Don't worry, I'm going to be adding the APIs for topology-level pausing as
part of the modular topologies KIP,
so we don't need to worry about that for now. That said, I don't think we
should brush it off entirely and design
this feature in a way that's going to be incompatible or hugely raise the
I think for named topology we can leave the scope of this KIP as "all or
nothing", i.e. when you pause an instance you pause all of its topologies.
I raised this question in my previous email just trying to clarify if this
is what you have in mind. We can leave the question of finer controlled
Hi Jim,
Thanks for the replies. This all sounds good to me. Just two further comments:
3. It seems like you should aim for the simplest semantics. If the intent is to
“pause” the instance, then you’d better pause the whole instance. If you leave
punctuations and standbys running, I expect we’d
Hi John,
Long emails are great; responding inline!
On Sat, May 7, 2022 at 4:54 PM John Roesler wrote:
> Thanks for the KIP, Jim!
>
> This conversation seems to highlight that the KIP needs to specify
> some of its behavior as well as its APIs, where the behavior is
> observable and significant
Thanks for the KIP, Jim!
This conversation seems to highlight that the KIP needs to specify
some of its behavior as well as its APIs, where the behavior is
observable and significant to users.
For example:
1. Do you plan to have a guarantee that immediately after
calling KafkaStreams.pause(),
I'm in favor of the "just pausing the instance itself“ option as well. As
for EOS, the point is that when the processing is paused, we would not
trigger any `producer.send` during the time, and the transaction timeout is
sort of relying on that behavior, so my point was that it's probably better
Hi Matthias,
Since the only thing which will be paused is processing the topology, I
think we can let commits happen naturally.
Good point about getting the paused state to new members; it is seeming
like the "building block" approach is a good one to keep things simple at
first.
Cheers,
Jim
I think it's tricky to propagate a pauseAll() via the rebalance
protocol. New members joining the group would need to get paused, too?
Could there be weird race conditions with overlapping pauseAll() and
resumeAll() calls on different instanced while there could be a errors /
network
Hi Bill,
Great questions; I'll do my best to reply inline:
On Fri, May 6, 2022 at 3:21 PM Bill Bejeck wrote:
> Hi Jim,
>
> Thanks for the KIP. I have a couple of meta-questions as well:
>
> 1) Regarding pausing only a subset of running instances, I'm thinking there
> may be a use case for
Hi Jim,
Thanks for the KIP. I have a couple of meta-questions as well:
1) Regarding pausing only a subset of running instances, I'm thinking there
may be a use case for pausing all of them.
Would it make sense to also allow for pausing all instances by adding a
method `pauseAll()` or
Hi Guozhang,
Thanks for the feedback; responses inline below:
On Fri, May 6, 2022 at 1:09 PM Guozhang Wang wrote:
> Hello Jim,
>
> Thanks for the proposed KIP. I have some meta questions about it:
>
> 1) Would an instance always pause/resume all of its current owned
> topologies (i.e. the
Hello Jim,
Thanks for the proposed KIP. I have some meta questions about it:
1) Would an instance always pause/resume all of its current owned
topologies (i.e. the named topologies), or are there any scenarios where we
only want to pause/resume a subset of them?
2) From a user's perspective, do
Hi all,
I have written up a KIP for adding the ability to pause and resume the
processing of a topology in AK Streams. The KIP is here:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=211882832
Thanks in advance for your feedback!
Cheers,
Jim
25 matches
Mail list logo