Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-30 Thread Colin McCabe
Thanks, Boyang. That seems reasonable. best, Colin On Mon, Mar 30, 2020, at 13:32, Boyang Chen wrote: > Thanks Guozhang for the summary. If everyone agrees to the strategy here, > which include: > > 1. No affection to 2.5 release, maintaining the same crashing logic for new > Producer API talki

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-30 Thread Boyang Chen
Thanks Guozhang for the summary. If everyone agrees to the strategy here, which include: 1. No affection to 2.5 release, maintaining the same crashing logic for new Producer API talking to old clients 2. Add an internal flag to silently downgrade protocol for Kafka Streams case only I will go ahe

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-30 Thread Guozhang Wang
Hello Colin, Thanks for the context you provided with CreateAclsRequest to make the point for compatibility policy, and I think they make sense -- given KIP-447 is primarily motivated for Streams clients, we should still consider it's potential compatibility pitfall for non-Streams clients. About

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-30 Thread Boyang Chen
Thanks for the thoughts everyone. Colin has made a good point about the supporting for non-stream applications, where silent downgrade is harmful and could make false promises to users. Matthias also made a good point about utilizing the internal flag to safe-guard in case non-stream users would al

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-29 Thread Colin McCabe
Thanks for the explanations, all. It seems to me that the proposed silent downgrade change leaves non-streams applications with no way to safely use KIP-447. If they use it, it will sometimes be safe, but sometimes not, depending on what they are doing. This doesn't seem acceptable. People s

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-28 Thread Matthias J. Sax
I agree with Guozhang that the issue only arises for the `read-process-write` pattern and thus we should guard at the read level already and thus auto-downgrading the write path commitTx request seems fine. Note that for producer-only and consumer-only apps the auto-downgrade is totally fine (and

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Guozhang Wang
Hello Colin, Ismael, Thanks for your feedbacks, they are quite helpful. Just to provide some context here about OffsetFetch: 1) When building the offset fetch request, we used to auto "downgrade" the request by falling back the requireStable flag when broker supporter version is < 7. https://git

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Ismael Juma
I'm a bit puzzled. We added this feature because we thought it was useful. And now we are saying that you don't know if you can rely on it since the downgrade happens silently. Can you provide more context on the OffsetFetch downgrade? What is the implication of that? Ismael On Fri, Mar 27, 2020

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Boyang Chen
Thanks Colin, I think the point of this change is to make the new client experience better while working with old brokers, when the upgrade is happening on client side only. For Streams, it is very common to have more advanced client version. On code level, the path to call transaction commit is be

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Colin McCabe
On Fri, Mar 27, 2020, at 18:29, Colin McCabe wrote: > On Fri, Mar 27, 2020, at 16:06, Boyang Chen wrote: > > Hey there, > > > > we would like to address an improvement on the > > *Producer#sendOffsetsToTransaction(offsets, > > groupMetadata) *API. Previously we assume the calling of this function

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Colin McCabe
On Fri, Mar 27, 2020, at 16:06, Boyang Chen wrote: > Hey there, > > we would like to address an improvement on the > *Producer#sendOffsetsToTransaction(offsets, > groupMetadata) *API. Previously we assume the calling of this function > would crash the app if broker is not on version 2.5.0 or highe

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-27 Thread Boyang Chen
Hey there, we would like to address an improvement on the *Producer#sendOffsetsToTransaction(offsets, groupMetadata) *API. Previously we assume the calling of this function would crash the app if broker is not on version 2.5.0 or higher. As Streams side change was rolling out, the disadvantage of

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-26 Thread Matthias J. Sax
One more change for KIP-447. Currently, Kafka Streams collects task-level metrics called "commit-latency-[max|avg]". However, with KIP-447 tasks don't get committed individually any longer, and thus this metrics do not make sense going forward. Therefore, we propose to remove those metrics in 2.6

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-03-05 Thread Matthias J. Sax
-BEGIN PGP SIGNED MESSAGE- Hash: SHA512 There is one more change to this KIP for the upgrade path of Kafka Streams applications: We cannot detect broker versions reliable, and thus, we need users to manually opt-in to the feature. Thus, we need to add a third value for configuration param

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-02-11 Thread Guozhang Wang
Boyang, Thanks for the update. This change makes sense to me. Guozhang On Tue, Feb 11, 2020 at 11:37 AM Boyang Chen wrote: > Hey there, > > we are adding a small change to the KIP-447 public API. The default value > of `transaction.abort.timed.out.transaction.cleanup.interval.ms` shall be > ch

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-02-11 Thread Boyang Chen
Hey there, we are adding a small change to the KIP-447 public API. The default value of `transaction.abort.timed.out.transaction.cleanup.interval.ms` shall be changed from 1 minute to 10 seconds. The goal here is to trigger the expired transaction more frequently in order to reduce the consumer pe

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-01-08 Thread Boyang Chen
Thanks Guozhang for another review! I have addressed all the javadoc changes for PendingTransactionException in the KIP. For FENCED_INSTANCE_ID the only thrown place would be on the new send offsets API, which is also addressed. Thanks Matthias for the vote! As we have 3 binding votes (Guozhang, J

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-01-08 Thread Matthias J. Sax
I just re-read the KIP. Overall I am +1 as well. Some minor comments (also apply to the Google design doc): 1) As 2.4 was release, references should be updated to 2.5. 2) About the upgrade path, the KIP says: 2a) > Broker must be upgraded to 2.4 first. This means the > `inter.broker.protoco

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2020-01-08 Thread Guozhang Wang
Hello Boyang, I've reviewed the open PRs you have created and also made another pass on the wiki. Here's some more comments: Meta: 1. From the code I realized that the newly introduced PendingTransactionException can be thrown out of the public APIs too (see my comments in the PR, e.g. from cons

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-18 Thread Boyang Chen
Bump this thread to see if someone could also review! On Mon, Sep 9, 2019 at 5:00 PM Boyang Chen wrote: > Thank you Jason! Addressed the comments. > > Thank you Guozhang for explaining. I will document the timeout setting > reasoning in the design doc. > > > On Mon, Sep 9, 2019 at 1:49 PM Guozha

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Boyang Chen
Thank you Jason! Addressed the comments. Thank you Guozhang for explaining. I will document the timeout setting reasoning in the design doc. On Mon, Sep 9, 2019 at 1:49 PM Guozhang Wang wrote: > On Fri, Sep 6, 2019 at 6:33 PM Boyang Chen > wrote: > > > Thanks Guozhang, I have polished the des

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Guozhang Wang
On Fri, Sep 6, 2019 at 6:33 PM Boyang Chen wrote: > Thanks Guozhang, I have polished the design doc to make it sync with > current KIP. As for overriding default timeout values, I guess it's already > stated in the KIP to set txn timeout to 10s, are you suggesting we should > also put down this r

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Jason Gustafson
> I agree group.instance.id should be nullable, for member.id, are you suggesting that the purpose is to be able to set it to null when we deserialize older version txn offset request? We will still support the old sendOffsetsToTransaction API, right? In that case, we won't have a memberId. On a

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Boyang Chen
Thanks Jason for the vote! On Mon, Sep 9, 2019 at 12:07 PM Jason Gustafson wrote: > +1 Thanks for the KIP. Just a couple comments below: > > 1. Kafka APIs traditionally leave off `get` from API names. How about > `groupMetadata` instead of `getMetadata`? > That sounds good to me. > 2. I am gues

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-09 Thread Jason Gustafson
+1 Thanks for the KIP. Just a couple comments below: 1. Kafka APIs traditionally leave off `get` from API names. How about `groupMetadata` instead of `getMetadata`? 2. I am guessing memberId and groupInstanceId should be nullable in the TxnOffsetCommit schema? 3. Just to clarify on the upgrade pro

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-06 Thread Boyang Chen
Thanks Guozhang, I have polished the design doc to make it sync with current KIP. As for overriding default timeout values, I guess it's already stated in the KIP to set txn timeout to 10s, are you suggesting we should also put down this recommendation on the KIP for non-stream EOS users? Boyang

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-05 Thread Guozhang Wang
Hello Boyang, Just realized one thing about timeout configurations that we should consider including in this KIP as well: 1) In Producer we have: max.block.ms (default value 60sec), request.timeout (30sec), delivery.timeout.ms (120sec), transaction.timeout (60sec) 2) In Consumer we have: session.

Re: [VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-05 Thread Guozhang Wang
Thanks Boyang, I'm +1 on the KIP. Could you also update the detailed design doc https://docs.google.com/document/d/1LhzHGeX7_Lay4xvrEXxfciuDWATjpUXQhrEIkph9qRE/edit which seems a bit out-dated with the latest proposal? Guozhang On Wed, Sep 4, 2019 at 10:45 AM Boyang Chen wrote: > Hey all, > >

[VOTE] KIP-447: Producer scalability for exactly once semantics

2019-09-04 Thread Boyang Chen
Hey all, I would like to start the vote for KIP-447 . This is a very important step to improve Kafka Streams scalability in exactly-once semantics, by avoiding linearly increasing number o