> 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 similar note, I guess the generationId
in this case would be -1?

> That's correct, I could add a side-note to the upgrade path, although I
think most people would like to pay the cost of
2 rolling bounces for the sake of consistency :)

You are more optimistic about this than I am ;). Anyway, I'm satisfied if
we just document the recommended approach and explain the risk if users
don't follow it.

-Jason

On Mon, Sep 9, 2019 at 12:52 PM Boyang Chen <reluctanthero...@gmail.com>
wrote:

> Thanks Jason for the vote!
>
> On Mon, Sep 9, 2019 at 12:07 PM Jason Gustafson <ja...@confluent.io>
> 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 guessing memberId and groupInstanceId should be nullable in the
> > TxnOffsetCommit schema?
> >
> 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?
>
> > 3. Just to clarify on the upgrade process for streams, the two-step
> update
> > is only required if you want strict guarantees about protection from
> > zombies. Is that right? So if you just do it in one shot, it will still
> > work, just you would be exposed to some edge cases.
> >
> That's correct, I could add a side-note to the upgrade path, although I
> think most people would like to pay the cost of
> 2 rolling bounces for the sake of consistency :)
>
>
> > Thanks,
> > Jason
> >
> > On Fri, Sep 6, 2019 at 6:33 PM Boyang Chen <reluctanthero...@gmail.com>
> > 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 recommendation on the KIP for non-stream EOS users?
> > >
> > > Boyang
> > >
> > > On Thu, Sep 5, 2019 at 8:43 PM Guozhang Wang <wangg...@gmail.com>
> wrote:
> > >
> > > > 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.timeout (10sec), request.timeout
> > (30sec),
> > > > default.api.timeout.ms (60sec).
> > > >
> > > > Within a transaction (i.e. after we've beginTxn), we could
> potentially
> > > call
> > > > consumer blocking APIs that depend on default.api.timeout.ms, and
> call
> > > > producer blocking APIs that depend on max.block.ms. Also, if the
> user
> > is
> > > > following a consumer->producer pattern, then it could be kicked and
> > > fenced
> > > > either by txn or by consumer group session.
> > > >
> > > > So we want to make sure that in the caller, e.g. Kafka Streams:
> > > >
> > > > 1) transaction.timeout < max.block.ms
> > > > 2) transaction.timeout < delivery.timeout.ms
> > > > 3) transaction.timeout < default.api.timeout.ms
> > > > 4) transaction.timeout ~= default.api.timeout.ms (I think this is
> > > already
> > > > mentioned in the KIP, just wanted to bring this up again)
> > > >
> > > > We do not need to override the default since not every users are
> > > following
> > > > the consumer -> producer pattern, but in cases like Streams where it
> is
> > > > indeed the case, we should override the default values to obey the
> > above
> > > > rules.
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Thu, Sep 5, 2019 at 5:47 PM Guozhang Wang <wangg...@gmail.com>
> > wrote:
> > > >
> > > > > 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 <
> > > reluctanthero...@gmail.com>
> > > > > wrote:
> > > > >
> > > > >> Hey all,
> > > > >>
> > > > >> I would like to start the vote for KIP-447
> > > > >> <
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics
> > > > >> >.
> > > > >> This is a very important step to improve Kafka Streams scalability
> > in
> > > > >> exactly-once semantics, by avoiding linearly increasing number of
> > > > >> producers
> > > > >> with topic partition increases.
> > > > >>
> > > > >> Thanks,
> > > > >> Boyang
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Reply via email to