You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Justine Olshan <jo...@confluent.io.INVALID> on 2024/03/19 01:39:57 UTC

Re: [DISCUSS] KIP-890 Server Side Defense

Hey there -- small update to the KIP,

The KIP mentioned introducing ABORTABLE_ERROR and bumping TxnOffsetCommit
and Produce requests. I've changed the name in the KIP to
ABORTABLE_TRANSACTION and the corresponding exception
AbortableTransactionException to match the pattern we had for other errors.
I also mentioned bumping all 6 transactional APIs so we can future
proof/support the error on the client going forward. If a future change
wants to have an error scenario that requires us to abort the transaction,
we can rely on the 3.8+ clients to support it. We ran into issues finding
good/generic error codes that older clients could support while working on
this KIP, so this should help in the future.

The features discussion is still ongoing in KIP-1022. Will update again
here when that concludes.

Justine

On Tue, Feb 6, 2024 at 8:39 AM Justine Olshan <jo...@confluent.io> wrote:

> I don't think AddPartitions is a good example since we currenly don't gate
> the version on TV or MV. (We only set a different flag depending on the TV)
>
> Even if we did want to gate it on TV, I think the idea is to move away
> from MV gating inter broker protocols. Ideally we can get to a state where
> MV is just used for metadata changes.
>
> I think some of this discussion might fit more with the feature version
> KIP, so I can try to open that up soon. Until we settle that, some of the
> work in KIP-890 is blocked.
>
> Justine
>
> On Mon, Feb 5, 2024 at 5:38 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>
>> Hi, Justine,
>>
>> Thanks for the reply.
>>
>> Since AddPartitions is an inter broker request, will its version be gated
>> only by TV or other features like MV too? For example, if we need to
>> change
>> the protocol for AddPartitions for reasons other than txn verification in
>> the future, will the new version be gated by a new MV? If so, does
>> downgrading a TV imply potential downgrade of MV too?
>>
>> Jun
>>
>>
>>
>> On Mon, Feb 5, 2024 at 5:07 PM Justine Olshan
>> <jo...@confluent.io.invalid>
>> wrote:
>>
>> > One TV gates the flexible feature version (no rpcs involved, only the
>> > transactional records that should only be gated by TV)
>> > Another TV gates the ability to turn on kip-890 part 2. This would gate
>> the
>> > version of Produce and EndTxn (likely only used by transactions), and
>> > specifies a flag in AddPartitionsToTxn though the version is already
>> used
>> > without TV.
>> >
>> > I think the only concern is the Produce request and we could consider
>> work
>> > arounds similar to the AddPartitionsToTxn call.
>> >
>> > Justine
>> >
>> > On Mon, Feb 5, 2024 at 4:56 PM Jun Rao <ju...@confluent.io.invalid>
>> wrote:
>> >
>> > > Hi, Justine,
>> > >
>> > > Which PRC/record protocols will TV guard? Going forward, will those
>> > > PRC/record protocols only be guarded by TV and not by other features
>> like
>> > > MV?
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > > On Mon, Feb 5, 2024 at 2:41 PM Justine Olshan
>> > <jolshan@confluent.io.invalid
>> > > >
>> > > wrote:
>> > >
>> > > > Hi Jun,
>> > > >
>> > > > Sorry I think I misunderstood your question or answered incorrectly.
>> > The
>> > > TV
>> > > > version should ideally be fully independent from MV.
>> > > > At least for the changes I proposed, TV should not affect MV and MV
>> > > should
>> > > > not affect TV/
>> > > >
>> > > > I think if we downgrade TV, only that feature should downgrade.
>> > Likewise
>> > > > the same with MV. The finalizedFeatures should just reflect the
>> feature
>> > > > downgrade we made.
>> > > >
>> > > > I also plan to write a new KIP for managing the disk format and
>> upgrade
>> > > > tool as we will need new flags to support these features. That
>> should
>> > > help
>> > > > clarify some things.
>> > > >
>> > > > Justine
>> > > >
>> > > > On Mon, Feb 5, 2024 at 11:03 AM Jun Rao <ju...@confluent.io.invalid>
>> > > wrote:
>> > > >
>> > > > > Hi, Justine,
>> > > > >
>> > > > > Thanks for the reply.
>> > > > >
>> > > > > So, if we downgrade TV, we could implicitly downgrade another
>> feature
>> > > > (say
>> > > > > MV) that has dependency (e.g. RPC). What would we return for
>> > > > > FinalizedFeatures for MV in ApiVersionsResponse in that case?
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > > On Fri, Feb 2, 2024 at 1:06 PM Justine Olshan
>> > > > <jolshan@confluent.io.invalid
>> > > > > >
>> > > > > wrote:
>> > > > >
>> > > > > > Hey Jun,
>> > > > > >
>> > > > > > Yes, the idea is that if we downgrade TV (transaction version)
>> we
>> > > will
>> > > > > stop
>> > > > > > using the add partitions to txn optimization and stop writing
>> the
>> > > > > flexible
>> > > > > > feature version of the log.
>> > > > > > In the compatibility section I included some explanations on how
>> > this
>> > > > is
>> > > > > > done.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Justine
>> > > > > >
>> > > > > > On Fri, Feb 2, 2024 at 11:12 AM Jun Rao
>> <ju...@confluent.io.invalid>
>> > > > > wrote:
>> > > > > >
>> > > > > > > Hi, Justine,
>> > > > > > >
>> > > > > > > Thanks for the update.
>> > > > > > >
>> > > > > > > If we ever downgrade the transaction feature, any feature
>> > depending
>> > > > on
>> > > > > > > changes on top of those RPC/record
>> > > > > > > (AddPartitionsToTxnRequest/TransactionLogValue) changes made
>> in
>> > > > KIP-890
>> > > > > > > will be automatically downgraded too?
>> > > > > > >
>> > > > > > > Jun
>> > > > > > >
>> > > > > > > On Tue, Jan 30, 2024 at 3:32 PM Justine Olshan
>> > > > > > > <jo...@confluent.io.invalid>
>> > > > > > > wrote:
>> > > > > > >
>> > > > > > > > Hey Jun,
>> > > > > > > >
>> > > > > > > > I wanted to get back to you about your questions about
>> MV/IBP.
>> > > > > > > >
>> > > > > > > > Looking at the options, I think it makes the most sense to
>> > > create a
>> > > > > > > > separate feature for transactions and use that to version
>> gate
>> > > the
>> > > > > > > features
>> > > > > > > > we need to version gate (flexible transactional state
>> records
>> > and
>> > > > > using
>> > > > > > > the
>> > > > > > > > new protocol)
>> > > > > > > > I've updated the KIP to include this change. Hopefully
>> that's
>> > > > > > everything
>> > > > > > > we
>> > > > > > > > need for this KIP :)
>> > > > > > > >
>> > > > > > > > Justine
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > On Mon, Jan 22, 2024 at 3:17 PM Justine Olshan <
>> > > > jolshan@confluent.io
>> > > > > >
>> > > > > > > > wrote:
>> > > > > > > >
>> > > > > > > > > Thanks Jun,
>> > > > > > > > >
>> > > > > > > > > I will update the KIP with the prev field for prepare as
>> > well.
>> > > > > > > > >
>> > > > > > > > > PREPARE
>> > > > > > > > > producerId: x
>> > > > > > > > > previous/lastProducerId (tagged field): x
>> > > > > > > > > nextProducerId (tagged field): empty or z if y will
>> overflow
>> > > > > > > > > producerEpoch: y + 1
>> > > > > > > > >
>> > > > > > > > > COMPLETE
>> > > > > > > > > producerId: x or z if y overflowed
>> > > > > > > > > previous/lastProducerId (tagged field): x
>> > > > > > > > > nextProducerId (tagged field): empty
>> > > > > > > > > producerEpoch: y + 1 or 0 if we overflowed
>> > > > > > > > >
>> > > > > > > > > Thanks again,
>> > > > > > > > > Justine
>> > > > > > > > >
>> > > > > > > > > On Mon, Jan 22, 2024 at 3:15 PM Jun Rao
>> > > <jun@confluent.io.invalid
>> > > > >
>> > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > >> Hi, Justine,
>> > > > > > > > >>
>> > > > > > > > >> 101.3 Thanks for the explanation.
>> > > > > > > > >> (1) My point was that the coordinator could fail right
>> after
>> > > > > writing
>> > > > > > > the
>> > > > > > > > >> prepare marker. When the new txn coordinator generates
>> the
>> > > > > complete
>> > > > > > > > marker
>> > > > > > > > >> after the failover, it needs some field from the prepare
>> > > marker
>> > > > to
>> > > > > > > > >> determine whether it's written by the new client.
>> > > > > > > > >>
>> > > > > > > > >> (2) The changing of the behavior sounds good to me. We
>> only
>> > > want
>> > > > > to
>> > > > > > > > return
>> > > > > > > > >> success if the prepare state is written by the new
>> client.
>> > So,
>> > > > in
>> > > > > > the
>> > > > > > > > >> non-overflow case, it seems that we also need sth in the
>> > > prepare
>> > > > > > > marker
>> > > > > > > > to
>> > > > > > > > >> tell us whether it's written by the new client.
>> > > > > > > > >>
>> > > > > > > > >> 112. Thanks for the explanation. That sounds good to me.
>> > > > > > > > >>
>> > > > > > > > >> Jun
>> > > > > > > > >>
>> > > > > > > > >> On Mon, Jan 22, 2024 at 11:32 AM Justine Olshan
>> > > > > > > > >> <jo...@confluent.io.invalid> wrote:
>> > > > > > > > >>
>> > > > > > > > >> > 101.3 I realized that I actually have two questions.
>> > > > > > > > >> > > (1) In the non-overflow case, we need to write the
>> > > previous
>> > > > > > > produce
>> > > > > > > > Id
>> > > > > > > > >> > tagged field in the end maker so that we know if the
>> > marker
>> > > is
>> > > > > > from
>> > > > > > > > the
>> > > > > > > > >> new
>> > > > > > > > >> > client. Since the end maker is derived from the prepare
>> > > > marker,
>> > > > > > > should
>> > > > > > > > >> we
>> > > > > > > > >> > write the previous produce Id in the prepare marker
>> field
>> > > too?
>> > > > > > > > >> Otherwise,
>> > > > > > > > >> > we will lose this information when deriving the end
>> > marker.
>> > > > > > > > >> >
>> > > > > > > > >> > The "previous" producer ID is in the normal producer ID
>> > > field.
>> > > > > So
>> > > > > > > yes,
>> > > > > > > > >> we
>> > > > > > > > >> > need it in prepare and that was always the plan.
>> > > > > > > > >> >
>> > > > > > > > >> > Maybe it is a bit unclear so I will enumerate the
>> fields
>> > and
>> > > > add
>> > > > > > > them
>> > > > > > > > to
>> > > > > > > > >> > the KIP if that helps.
>> > > > > > > > >> > Say we have producer ID x and epoch y. When we overflow
>> > > epoch
>> > > > y
>> > > > > we
>> > > > > > > get
>> > > > > > > > >> > producer ID Z.
>> > > > > > > > >> >
>> > > > > > > > >> > PREPARE
>> > > > > > > > >> > producerId: x
>> > > > > > > > >> > previous/lastProducerId (tagged field): empty
>> > > > > > > > >> > nextProducerId (tagged field): empty or z if y will
>> > overflow
>> > > > > > > > >> > producerEpoch: y + 1
>> > > > > > > > >> >
>> > > > > > > > >> > COMPLETE
>> > > > > > > > >> > producerId: x or z if y overflowed
>> > > > > > > > >> > previous/lastProducerId (tagged field): x
>> > > > > > > > >> > nextProducerId (tagged field): empty
>> > > > > > > > >> > producerEpoch: y + 1 or 0 if we overflowed
>> > > > > > > > >> >
>> > > > > > > > >> > (2) In the prepare phase, if we retry and see epoch -
>> 1 +
>> > ID
>> > > > in
>> > > > > > last
>> > > > > > > > >> seen
>> > > > > > > > >> > fields and are issuing the same command (ie commit not
>> > > abort),
>> > > > > we
>> > > > > > > > return
>> > > > > > > > >> > success. The logic before KIP-890 seems to return
>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>> > > > > > > > >> > in this case. Are we intentionally making this change?
>> > > > > > > > >> >
>> > > > > > > > >> > Hmm -- we would fence the producer if the epoch is
>> bumped
>> > > and
>> > > > we
>> > > > > > > get a
>> > > > > > > > >> > lower epoch. Yes -- we are intentionally adding this to
>> > > > prevent
>> > > > > > > > fencing.
>> > > > > > > > >> >
>> > > > > > > > >> >
>> > > > > > > > >> > 112. We already merged the code that adds the
>> VerifyOnly
>> > > field
>> > > > > in
>> > > > > > > > >> > AddPartitionsToTxnRequest, which is an inter broker
>> > request.
>> > > > It
>> > > > > > > seems
>> > > > > > > > >> that
>> > > > > > > > >> > we didn't bump up the IBP for that. Do you know why?
>> > > > > > > > >> >
>> > > > > > > > >> > We no longer need IBP for all interbroker requests as
>> > > > > ApiVersions
>> > > > > > > > should
>> > > > > > > > >> > correctly gate versioning.
>> > > > > > > > >> > We also handle unsupported version errors correctly if
>> we
>> > > > > receive
>> > > > > > > them
>> > > > > > > > >> in
>> > > > > > > > >> > edge cases like upgrades/downgrades.
>> > > > > > > > >> >
>> > > > > > > > >> > Justine
>> > > > > > > > >> >
>> > > > > > > > >> > On Mon, Jan 22, 2024 at 11:00 AM Jun Rao
>> > > > > <jun@confluent.io.invalid
>> > > > > > >
>> > > > > > > > >> wrote:
>> > > > > > > > >> >
>> > > > > > > > >> > > Hi, Justine,
>> > > > > > > > >> > >
>> > > > > > > > >> > > Thanks for the reply.
>> > > > > > > > >> > >
>> > > > > > > > >> > > 101.3 I realized that I actually have two questions.
>> > > > > > > > >> > > (1) In the non-overflow case, we need to write the
>> > > previous
>> > > > > > > produce
>> > > > > > > > Id
>> > > > > > > > >> > > tagged field in the end maker so that we know if the
>> > > marker
>> > > > is
>> > > > > > > from
>> > > > > > > > >> the
>> > > > > > > > >> > new
>> > > > > > > > >> > > client. Since the end maker is derived from the
>> prepare
>> > > > > marker,
>> > > > > > > > >> should we
>> > > > > > > > >> > > write the previous produce Id in the prepare marker
>> > field
>> > > > too?
>> > > > > > > > >> Otherwise,
>> > > > > > > > >> > > we will lose this information when deriving the end
>> > > marker.
>> > > > > > > > >> > > (2) In the prepare phase, if we retry and see epoch
>> - 1
>> > +
>> > > ID
>> > > > > in
>> > > > > > > last
>> > > > > > > > >> seen
>> > > > > > > > >> > > fields and are issuing the same command (ie commit
>> not
>> > > > abort),
>> > > > > > we
>> > > > > > > > >> return
>> > > > > > > > >> > > success. The logic before KIP-890 seems to return
>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>> > > > > > > > >> > > in this case. Are we intentionally making this
>> change?
>> > > > > > > > >> > >
>> > > > > > > > >> > > 112. We already merged the code that adds the
>> VerifyOnly
>> > > > field
>> > > > > > in
>> > > > > > > > >> > > AddPartitionsToTxnRequest, which is an inter broker
>> > > request.
>> > > > > It
>> > > > > > > > seems
>> > > > > > > > >> > that
>> > > > > > > > >> > > we didn't bump up the IBP for that. Do you know why?
>> > > > > > > > >> > >
>> > > > > > > > >> > > Jun
>> > > > > > > > >> > >
>> > > > > > > > >> > > On Fri, Jan 19, 2024 at 4:50 PM Justine Olshan
>> > > > > > > > >> > > <jo...@confluent.io.invalid>
>> > > > > > > > >> > > wrote:
>> > > > > > > > >> > >
>> > > > > > > > >> > > > Hi Jun,
>> > > > > > > > >> > > >
>> > > > > > > > >> > > > 101.3 I can change "last seen" to "current
>> producer id
>> > > and
>> > > > > > > epoch"
>> > > > > > > > if
>> > > > > > > > >> > that
>> > > > > > > > >> > > > was the part that was confusing
>> > > > > > > > >> > > > 110 I can mention this
>> > > > > > > > >> > > > 111 I can do that
>> > > > > > > > >> > > > 112 We still need it. But I am still finalizing the
>> > > > design.
>> > > > > I
>> > > > > > > will
>> > > > > > > > >> > update
>> > > > > > > > >> > > > the KIP once I get the information finalized. Sorry
>> > for
>> > > > the
>> > > > > > > > delays.
>> > > > > > > > >> > > >
>> > > > > > > > >> > > > Justine
>> > > > > > > > >> > > >
>> > > > > > > > >> > > > On Fri, Jan 19, 2024 at 10:50 AM Jun Rao
>> > > > > > > <jun@confluent.io.invalid
>> > > > > > > > >
>> > > > > > > > >> > > wrote:
>> > > > > > > > >> > > >
>> > > > > > > > >> > > > > Hi, Justine,
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > Thanks for the reply.
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > 101.3 In the non-overflow case, the previous ID
>> is
>> > the
>> > > > > same
>> > > > > > as
>> > > > > > > > the
>> > > > > > > > >> > > > produce
>> > > > > > > > >> > > > > ID for the complete marker too, but we set the
>> > > previous
>> > > > ID
>> > > > > > in
>> > > > > > > > the
>> > > > > > > > >> > > > complete
>> > > > > > > > >> > > > > marker. Earlier you mentioned that this is to
>> know
>> > > that
>> > > > > the
>> > > > > > > > >> marker is
>> > > > > > > > >> > > > > written by the new client so that we could return
>> > > > success
>> > > > > on
>> > > > > > > > >> retried
>> > > > > > > > >> > > > > endMarker requests. I was trying to understand
>> why
>> > > this
>> > > > is
>> > > > > > not
>> > > > > > > > >> needed
>> > > > > > > > >> > > for
>> > > > > > > > >> > > > > the prepare marker since retry can happen in the
>> > > prepare
>> > > > > > state
>> > > > > > > > >> too.
>> > > > > > > > >> > Is
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > reason that in the prepare state, we return
>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>> > > > > > > > >> > > > instead
>> > > > > > > > >> > > > > of success on retried endMaker requests? If so,
>> > should
>> > > > we
>> > > > > > > change
>> > > > > > > > >> "If
>> > > > > > > > >> > we
>> > > > > > > > >> > > > > retry and see epoch - 1 + ID in last seen fields
>> and
>> > > are
>> > > > > > > issuing
>> > > > > > > > >> the
>> > > > > > > > >> > > same
>> > > > > > > > >> > > > > command (ie commit not abort) we can return (with
>> > the
>> > > > new
>> > > > > > > > epoch)"
>> > > > > > > > >> > > > > accordingly?
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > 110. Yes, without this KIP, a delayed endMaker
>> > request
>> > > > > > carries
>> > > > > > > > the
>> > > > > > > > >> > same
>> > > > > > > > >> > > > > epoch and won't be fenced. This can commit/abort
>> a
>> > > > future
>> > > > > > > > >> transaction
>> > > > > > > > >> > > > > unexpectedly. I am not sure if we have seen this
>> in
>> > > > > practice
>> > > > > > > > >> though.
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > 111. Sounds good. It would be useful to make it
>> > clear
>> > > > that
>> > > > > > we
>> > > > > > > > can
>> > > > > > > > >> now
>> > > > > > > > >> > > > > populate the lastSeen field from the log
>> reliably.
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > 112. Yes, I was referring to
>> > AddPartitionsToTxnRequest
>> > > > > since
>> > > > > > > > it's
>> > > > > > > > >> > > called
>> > > > > > > > >> > > > > across brokers and we are changing its schema.
>> Are
>> > you
>> > > > > > saying
>> > > > > > > we
>> > > > > > > > >> > don't
>> > > > > > > > >> > > > need
>> > > > > > > > >> > > > > it any more? I thought that we already
>> implemented
>> > the
>> > > > > > server
>> > > > > > > > side
>> > > > > > > > >> > > > > verification logic based on
>> > AddPartitionsToTxnRequest
>> > > > > across
>> > > > > > > > >> brokers.
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > Jun
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > On Thu, Jan 18, 2024 at 5:05 PM Justine Olshan
>> > > > > > > > >> > > > > <jo...@confluent.io.invalid>
>> > > > > > > > >> > > > > wrote:
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > > > > Hey Jun,
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > 101.3 We don't set the previous ID in the
>> Prepare
>> > > > field
>> > > > > > > since
>> > > > > > > > we
>> > > > > > > > >> > > don't
>> > > > > > > > >> > > > > need
>> > > > > > > > >> > > > > > it. It is the same producer ID as the main
>> > producer
>> > > ID
>> > > > > > > field.
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > 110 Hmm -- maybe I need to reread your message
>> > about
>> > > > > > delayed
>> > > > > > > > >> > markers.
>> > > > > > > > >> > > > If
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > receive a delayed endTxn marker after the
>> > > transaction
>> > > > is
>> > > > > > > > already
>> > > > > > > > >> > > > > complete?
>> > > > > > > > >> > > > > > So we will commit the next transaction early
>> > without
>> > > > the
>> > > > > > > fixes
>> > > > > > > > >> in
>> > > > > > > > >> > > part
>> > > > > > > > >> > > > 2?
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > 111 Yes -- this terminology was used in a
>> previous
>> > > KIP
>> > > > > and
>> > > > > > > > never
>> > > > > > > > >> > > > > > implemented it in the log -- only in memory
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > 112 Hmm -- which interbroker protocol are you
>> > > > referring
>> > > > > > to?
>> > > > > > > I
>> > > > > > > > am
>> > > > > > > > >> > > > working
>> > > > > > > > >> > > > > on
>> > > > > > > > >> > > > > > the design for the work to remove the extra add
>> > > > > partitions
>> > > > > > > > call
>> > > > > > > > >> > and I
>> > > > > > > > >> > > > > right
>> > > > > > > > >> > > > > > now the design bumps MV. I have yet to update
>> that
>> > > > > section
>> > > > > > > as
>> > > > > > > > I
>> > > > > > > > >> > > > finalize
>> > > > > > > > >> > > > > > the design so please stay tuned. Was there
>> > anything
>> > > > else
>> > > > > > you
>> > > > > > > > >> > thought
>> > > > > > > > >> > > > > needed
>> > > > > > > > >> > > > > > MV bump?
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > Justine
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > On Thu, Jan 18, 2024 at 3:07 PM Jun Rao
>> > > > > > > > >> <ju...@confluent.io.invalid>
>> > > > > > > > >> > > > > wrote:
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > I don't see this create any issue. It just
>> makes
>> > > it
>> > > > a
>> > > > > > bit
>> > > > > > > > >> hard to
>> > > > > > > > >> > > > > explain
>> > > > > > > > >> > > > > > > what this non-tagged produce id field means.
>> We
>> > > are
>> > > > > > > > >> essentially
>> > > > > > > > >> > > > trying
>> > > > > > > > >> > > > > to
>> > > > > > > > >> > > > > > > combine two actions (completing a txn and
>> init a
>> > > new
>> > > > > > > produce
>> > > > > > > > >> Id)
>> > > > > > > > >> > > in a
>> > > > > > > > >> > > > > > > single record. But, this may be fine too.
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > A few other follow up comments.
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > 101.3 I guess the reason that we only set the
>> > > > previous
>> > > > > > > > >> produce id
>> > > > > > > > >> > > > > tagged
>> > > > > > > > >> > > > > > > field in the complete marker, but not in the
>> > > prepare
>> > > > > > > marker,
>> > > > > > > > >> is
>> > > > > > > > >> > > that
>> > > > > > > > >> > > > in
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > prepare state, we always return
>> > > > > CONCURRENT_TRANSACTIONS
>> > > > > > on
>> > > > > > > > >> > retried
>> > > > > > > > >> > > > > > endMaker
>> > > > > > > > >> > > > > > > requests?
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > 110. "I believe your second point is
>> mentioned
>> > in
>> > > > the
>> > > > > > > KIP. I
>> > > > > > > > >> can
>> > > > > > > > >> > > add
>> > > > > > > > >> > > > > more
>> > > > > > > > >> > > > > > > text on
>> > > > > > > > >> > > > > > > this if it is helpful.
>> > > > > > > > >> > > > > > > > The delayed message case can also violate
>> EOS
>> > if
>> > > > the
>> > > > > > > > delayed
>> > > > > > > > >> > > > message
>> > > > > > > > >> > > > > > > comes in after the next addPartitionsToTxn
>> > request
>> > > > > comes
>> > > > > > > in.
>> > > > > > > > >> > > > > Effectively
>> > > > > > > > >> > > > > > we
>> > > > > > > > >> > > > > > > may see a message from a previous (aborted)
>> > > > > transaction
>> > > > > > > > become
>> > > > > > > > >> > part
>> > > > > > > > >> > > > of
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > next transaction."
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > The above is the case when a delayed message
>> is
>> > > > > appended
>> > > > > > > to
>> > > > > > > > >> the
>> > > > > > > > >> > > data
>> > > > > > > > >> > > > > > > partition. What I mentioned is a slightly
>> > > different
>> > > > > case
>> > > > > > > > when
>> > > > > > > > >> a
>> > > > > > > > >> > > > delayed
>> > > > > > > > >> > > > > > > marker is appended to the transaction log
>> > > partition.
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > 111. The KIP says "Once we move past the
>> Prepare
>> > > and
>> > > > > > > > Complete
>> > > > > > > > >> > > states,
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > > don’t need to worry about lastSeen fields and
>> > > clear
>> > > > > > them,
>> > > > > > > > just
>> > > > > > > > >> > > handle
>> > > > > > > > >> > > > > > state
>> > > > > > > > >> > > > > > > transitions as normal.". Is the lastSeen
>> field
>> > the
>> > > > > same
>> > > > > > as
>> > > > > > > > the
>> > > > > > > > >> > > > previous
>> > > > > > > > >> > > > > > > Produce Id tagged field in
>> TransactionLogValue?
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > 112. Since the kip changes the inter-broker
>> > > > protocol,
>> > > > > > > should
>> > > > > > > > >> we
>> > > > > > > > >> > > bump
>> > > > > > > > >> > > > up
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > MV/IBP version? Is this feature only for the
>> > KRaft
>> > > > > mode?
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > Thanks,
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > Jun
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > On Wed, Jan 17, 2024 at 11:13 AM Justine
>> Olshan
>> > > > > > > > >> > > > > > > <jo...@confluent.io.invalid> wrote:
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > > > > Hey Jun,
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > > > I'm glad we are getting to convergence on
>> the
>> > > > > design.
>> > > > > > :)
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > > > While I understand it seems a little
>> "weird".
>> > > I'm
>> > > > > not
>> > > > > > > sure
>> > > > > > > > >> what
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > benefit
>> > > > > > > > >> > > > > > > > of writing an extra record to the log.
>> > > > > > > > >> > > > > > > > Is the concern a tool to describe
>> transactions
>> > > > won't
>> > > > > > > work
>> > > > > > > > >> (ie,
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > complete
>> > > > > > > > >> > > > > > > > state is needed to calculate the time since
>> > the
>> > > > > > > > transaction
>> > > > > > > > >> > > > > completed?)
>> > > > > > > > >> > > > > > > > If we have a reason like this, it is
>> enough to
>> > > > > > convince
>> > > > > > > me
>> > > > > > > > >> we
>> > > > > > > > >> > > need
>> > > > > > > > >> > > > > such
>> > > > > > > > >> > > > > > > an
>> > > > > > > > >> > > > > > > > extra record. It seems like it would be
>> > > replacing
>> > > > > the
>> > > > > > > > record
>> > > > > > > > >> > > > written
>> > > > > > > > >> > > > > on
>> > > > > > > > >> > > > > > > > InitProducerId. Is this correct?
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > > > Thanks,
>> > > > > > > > >> > > > > > > > Justine
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > > > On Tue, Jan 16, 2024 at 5:14 PM Jun Rao
>> > > > > > > > >> > <jun@confluent.io.invalid
>> > > > > > > > >> > > >
>> > > > > > > > >> > > > > > > wrote:
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > Thanks for the explanation. I understand
>> the
>> > > > > > intention
>> > > > > > > > >> now.
>> > > > > > > > >> > In
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > overflow
>> > > > > > > > >> > > > > > > > > case, we set the non-tagged field to the
>> old
>> > > pid
>> > > > > > (and
>> > > > > > > > the
>> > > > > > > > >> max
>> > > > > > > > >> > > > > epoch)
>> > > > > > > > >> > > > > > in
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > prepare marker so that we could correctly
>> > > write
>> > > > > the
>> > > > > > > > >> marker to
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > data
>> > > > > > > > >> > > > > > > > > partition if the broker downgrades. When
>> > > writing
>> > > > > the
>> > > > > > > > >> complete
>> > > > > > > > >> > > > > marker,
>> > > > > > > > >> > > > > > > we
>> > > > > > > > >> > > > > > > > > know the marker has already been written
>> to
>> > > the
>> > > > > data
>> > > > > > > > >> > partition.
>> > > > > > > > >> > > > We
>> > > > > > > > >> > > > > > set
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > non-tagged field to the new pid to avoid
>> > > > > > > > >> > > > InvalidPidMappingException
>> > > > > > > > >> > > > > > in
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > client if the broker downgrades.
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > The above seems to work. It's just a bit
>> > > > > > inconsistent
>> > > > > > > > for
>> > > > > > > > >> a
>> > > > > > > > >> > > > prepare
>> > > > > > > > >> > > > > > > > marker
>> > > > > > > > >> > > > > > > > > and a complete marker to use different
>> pids
>> > in
>> > > > > this
>> > > > > > > > >> special
>> > > > > > > > >> > > case.
>> > > > > > > > >> > > > > If
>> > > > > > > > >> > > > > > we
>> > > > > > > > >> > > > > > > > > downgrade with the complete marker, it
>> seems
>> > > > that
>> > > > > we
>> > > > > > > > will
>> > > > > > > > >> > never
>> > > > > > > > >> > > > be
>> > > > > > > > >> > > > > > able
>> > > > > > > > >> > > > > > > > to
>> > > > > > > > >> > > > > > > > > write the complete marker with the old
>> pid.
>> > > Not
>> > > > > sure
>> > > > > > > if
>> > > > > > > > it
>> > > > > > > > >> > > causes
>> > > > > > > > >> > > > > any
>> > > > > > > > >> > > > > > > > > issue, but it seems a bit weird. Instead
>> of
>> > > > > writing
>> > > > > > > the
>> > > > > > > > >> > > complete
>> > > > > > > > >> > > > > > marker
>> > > > > > > > >> > > > > > > > > with the new pid, could we write two
>> > records:
>> > > a
>> > > > > > > complete
>> > > > > > > > >> > marker
>> > > > > > > > >> > > > > with
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > old pid followed by a TransactionLogValue
>> > with
>> > > > the
>> > > > > > new
>> > > > > > > > pid
>> > > > > > > > >> > and
>> > > > > > > > >> > > an
>> > > > > > > > >> > > > > > empty
>> > > > > > > > >> > > > > > > > > state? We could make the two records in
>> the
>> > > same
>> > > > > > batch
>> > > > > > > > so
>> > > > > > > > >> > that
>> > > > > > > > >> > > > they
>> > > > > > > > >> > > > > > > will
>> > > > > > > > >> > > > > > > > be
>> > > > > > > > >> > > > > > > > > added to the log atomically.
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > Thanks,
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > Jun
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > On Fri, Jan 12, 2024 at 5:40 PM Justine
>> > Olshan
>> > > > > > > > >> > > > > > > > > <jo...@confluent.io.invalid>
>> > > > > > > > >> > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > > (1) the prepare marker is written, but
>> the
>> > > > > endTxn
>> > > > > > > > >> response
>> > > > > > > > >> > is
>> > > > > > > > >> > > > not
>> > > > > > > > >> > > > > > > > > received
>> > > > > > > > >> > > > > > > > > > by the client when the server
>> downgrades
>> > > > > > > > >> > > > > > > > > > (2)  the prepare marker is written, the
>> > > endTxn
>> > > > > > > > response
>> > > > > > > > >> is
>> > > > > > > > >> > > > > received
>> > > > > > > > >> > > > > > > by
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > client when the server downgrades.
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > I think I am still a little confused.
>> In
>> > > both
>> > > > of
>> > > > > > > these
>> > > > > > > > >> > cases,
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > > > transaction log has the old producer
>> ID.
>> > We
>> > > > > don't
>> > > > > > > > write
>> > > > > > > > >> the
>> > > > > > > > >> > > new
>> > > > > > > > >> > > > > > > > producer
>> > > > > > > > >> > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > in the prepare marker's non tagged
>> fields.
>> > > > > > > > >> > > > > > > > > > If the server downgrades now, it would
>> > read
>> > > > the
>> > > > > > > > records
>> > > > > > > > >> not
>> > > > > > > > >> > > in
>> > > > > > > > >> > > > > > tagged
>> > > > > > > > >> > > > > > > > > > fields and the complete marker will
>> also
>> > > have
>> > > > > the
>> > > > > > > old
>> > > > > > > > >> > > producer
>> > > > > > > > >> > > > > ID.
>> > > > > > > > >> > > > > > > > > > (If we had used the new producer ID, we
>> > > would
>> > > > > not
>> > > > > > > have
>> > > > > > > > >> > > > > > transactional
>> > > > > > > > >> > > > > > > > > > correctness since the producer id
>> doesn't
>> > > > match
>> > > > > > the
>> > > > > > > > >> > > transaction
>> > > > > > > > >> > > > > and
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > > state would not be correct on the data
>> > > > > partition.)
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > In the overflow case, I'd expect the
>> > > following
>> > > > > to
>> > > > > > > > >> happen on
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > client
>> > > > > > > > >> > > > > > > > > side
>> > > > > > > > >> > > > > > > > > > Case 1  -- we retry EndTxn -- it is the
>> > same
>> > > > > > > producer
>> > > > > > > > ID
>> > > > > > > > >> > and
>> > > > > > > > >> > > > > epoch
>> > > > > > > > >> > > > > > -
>> > > > > > > > >> > > > > > > 1
>> > > > > > > > >> > > > > > > > > this
>> > > > > > > > >> > > > > > > > > > would fence the producer
>> > > > > > > > >> > > > > > > > > > Case 2 -- we don't retry EndTxn and use
>> > the
>> > > > new
>> > > > > > > > >> producer id
>> > > > > > > > >> > > > which
>> > > > > > > > >> > > > > > > would
>> > > > > > > > >> > > > > > > > > > result in InvalidPidMappingException
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > Maybe we can have special handling for
>> > when
>> > > a
>> > > > > > server
>> > > > > > > > >> > > > downgrades.
>> > > > > > > > >> > > > > > When
>> > > > > > > > >> > > > > > > > it
>> > > > > > > > >> > > > > > > > > > reconnects we could get an API version
>> > > request
>> > > > > > > showing
>> > > > > > > > >> > > KIP-890
>> > > > > > > > >> > > > > > part 2
>> > > > > > > > >> > > > > > > > is
>> > > > > > > > >> > > > > > > > > > not supported. In that case, we can
>> call
>> > > > > > > > initProducerId
>> > > > > > > > >> to
>> > > > > > > > >> > > > abort
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > > > transaction. (In the overflow case,
>> this
>> > > > > correctly
>> > > > > > > > gives
>> > > > > > > > >> > us a
>> > > > > > > > >> > > > new
>> > > > > > > > >> > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > ID)
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > I guess the corresponding case would be
>> > > where
>> > > > > the
>> > > > > > > > >> *complete
>> > > > > > > > >> > > > > marker
>> > > > > > > > >> > > > > > > *is
>> > > > > > > > >> > > > > > > > > > written but the endTxn is not received
>> by
>> > > the
>> > > > > > client
>> > > > > > > > and
>> > > > > > > > >> > the
>> > > > > > > > >> > > > > server
>> > > > > > > > >> > > > > > > > > > downgrades? This would result in the
>> > > > transaction
>> > > > > > > > >> > coordinator
>> > > > > > > > >> > > > > having
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > new
>> > > > > > > > >> > > > > > > > > > ID and not the old one.  If the client
>> > > > retries,
>> > > > > it
>> > > > > > > > will
>> > > > > > > > >> > > receive
>> > > > > > > > >> > > > > an
>> > > > > > > > >> > > > > > > > > > InvalidPidMappingException. The
>> > > InitProducerId
>> > > > > > > > scenario
>> > > > > > > > >> > above
>> > > > > > > > >> > > > > would
>> > > > > > > > >> > > > > > > > help
>> > > > > > > > >> > > > > > > > > > here too.
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > To be clear, my compatibility story is
>> > meant
>> > > > to
>> > > > > > > > support
>> > > > > > > > >> > > > > downgrades
>> > > > > > > > >> > > > > > > > server
>> > > > > > > > >> > > > > > > > > > side in keeping the transactional
>> > > correctness.
>> > > > > > > Keeping
>> > > > > > > > >> the
>> > > > > > > > >> > > > client
>> > > > > > > > >> > > > > > > from
>> > > > > > > > >> > > > > > > > > > fencing itself is not the priority.
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > Hope this helps. I can also add text in
>> > the
>> > > > KIP
>> > > > > > > about
>> > > > > > > > >> > > > > > InitProducerId
>> > > > > > > > >> > > > > > > if
>> > > > > > > > >> > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > think that fixes some edge cases.
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > Justine
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > On Fri, Jan 12, 2024 at 4:10 PM Jun Rao
>> > > > > > > > >> > > > <jun@confluent.io.invalid
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > Thanks for the reply.
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > I agree that we don't need to
>> optimize
>> > for
>> > > > > > fencing
>> > > > > > > > >> during
>> > > > > > > > >> > > > > > > downgrades.
>> > > > > > > > >> > > > > > > > > > > Regarding consistency, there are two
>> > > > possible
>> > > > > > > cases:
>> > > > > > > > >> (1)
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > prepare
>> > > > > > > > >> > > > > > > > > > marker
>> > > > > > > > >> > > > > > > > > > > is written, but the endTxn response
>> is
>> > not
>> > > > > > > received
>> > > > > > > > by
>> > > > > > > > >> > the
>> > > > > > > > >> > > > > client
>> > > > > > > > >> > > > > > > > when
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > server downgrades; (2)  the prepare
>> > marker
>> > > > is
>> > > > > > > > written,
>> > > > > > > > >> > the
>> > > > > > > > >> > > > > endTxn
>> > > > > > > > >> > > > > > > > > > response
>> > > > > > > > >> > > > > > > > > > > is received by the client when the
>> > server
>> > > > > > > > downgrades.
>> > > > > > > > >> In
>> > > > > > > > >> > > (1),
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > > client
>> > > > > > > > >> > > > > > > > > > > will have the old produce Id and in
>> (2),
>> > > the
>> > > > > > > client
>> > > > > > > > >> will
>> > > > > > > > >> > > have
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > new
>> > > > > > > > >> > > > > > > > > > > produce Id. If we downgrade right
>> after
>> > > the
>> > > > > > > prepare
>> > > > > > > > >> > marker,
>> > > > > > > > >> > > > we
>> > > > > > > > >> > > > > > > can't
>> > > > > > > > >> > > > > > > > be
>> > > > > > > > >> > > > > > > > > > > consistent to both (1) and (2) since
>> we
>> > > can
>> > > > > only
>> > > > > > > put
>> > > > > > > > >> one
>> > > > > > > > >> > > > value
>> > > > > > > > >> > > > > in
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > > > existing produce Id field. It's also
>> not
>> > > > clear
>> > > > > > > which
>> > > > > > > > >> case
>> > > > > > > > >> > > is
>> > > > > > > > >> > > > > more
>> > > > > > > > >> > > > > > > > > likely.
>> > > > > > > > >> > > > > > > > > > > So we could probably be consistent
>> with
>> > > > either
>> > > > > > > case.
>> > > > > > > > >> By
>> > > > > > > > >> > > > putting
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > new
>> > > > > > > > >> > > > > > > > > > > producer Id in the prepare marker, we
>> > are
>> > > > > > > consistent
>> > > > > > > > >> with
>> > > > > > > > >> > > > case
>> > > > > > > > >> > > > > > (2)
>> > > > > > > > >> > > > > > > > and
>> > > > > > > > >> > > > > > > > > it
>> > > > > > > > >> > > > > > > > > > > also has the slight benefit that the
>> > > produce
>> > > > > > field
>> > > > > > > > in
>> > > > > > > > >> the
>> > > > > > > > >> > > > > prepare
>> > > > > > > > >> > > > > > > and
>> > > > > > > > >> > > > > > > > > > > complete marker are consistent in the
>> > > > overflow
>> > > > > > > case.
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > Jun
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > On Fri, Jan 12, 2024 at 3:11 PM
>> Justine
>> > > > Olshan
>> > > > > > > > >> > > > > > > > > > > <jo...@confluent.io.invalid>
>> > > > > > > > >> > > > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > Hi Jun,
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > In the case you describe, we would
>> > need
>> > > to
>> > > > > > have
>> > > > > > > a
>> > > > > > > > >> > delayed
>> > > > > > > > >> > > > > > > request,
>> > > > > > > > >> > > > > > > > > > send a
>> > > > > > > > >> > > > > > > > > > > > successful EndTxn, and a successful
>> > > > > > > > >> AddPartitionsToTxn
>> > > > > > > > >> > > and
>> > > > > > > > >> > > > > then
>> > > > > > > > >> > > > > > > > have
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > delayed EndTxn request go through
>> for
>> > a
>> > > > > given
>> > > > > > > > >> producer.
>> > > > > > > > >> > > > > > > > > > > > I'm trying to figure out if it is
>> > > possible
>> > > > > for
>> > > > > > > the
>> > > > > > > > >> > client
>> > > > > > > > >> > > > to
>> > > > > > > > >> > > > > > > > > transition
>> > > > > > > > >> > > > > > > > > > > if
>> > > > > > > > >> > > > > > > > > > > > a previous request is delayed
>> > somewhere.
>> > > > But
>> > > > > > > yes,
>> > > > > > > > in
>> > > > > > > > >> > this
>> > > > > > > > >> > > > > case
>> > > > > > > > >> > > > > > I
>> > > > > > > > >> > > > > > > > > think
>> > > > > > > > >> > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > would fence the client.
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > Not for the overflow case. In the
>> > > overflow
>> > > > > > case,
>> > > > > > > > the
>> > > > > > > > >> > > > producer
>> > > > > > > > >> > > > > > ID
>> > > > > > > > >> > > > > > > > and
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > epoch are different on the marker
>> and
>> > on
>> > > > the
>> > > > > > new
>> > > > > > > > >> > > > transaction.
>> > > > > > > > >> > > > > > So
>> > > > > > > > >> > > > > > > we
>> > > > > > > > >> > > > > > > > > > want
>> > > > > > > > >> > > > > > > > > > > > the marker to use the max epoch
>> but
>> > the
>> > > > new
>> > > > > > > > >> > transaction
>> > > > > > > > >> > > > > should
>> > > > > > > > >> > > > > > > > start
>> > > > > > > > >> > > > > > > > > > > with
>> > > > > > > > >> > > > > > > > > > > > the new ID and epoch 0 in the
>> > > > transactional
>> > > > > > > state.
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > In the server downgrade case, we
>> want
>> > to
>> > > > see
>> > > > > > the
>> > > > > > > > >> > producer
>> > > > > > > > >> > > > ID
>> > > > > > > > >> > > > > as
>> > > > > > > > >> > > > > > > > that
>> > > > > > > > >> > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > what the client will have. If we
>> > > complete
>> > > > > the
>> > > > > > > > >> commit,
>> > > > > > > > >> > and
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > > > transaction
>> > > > > > > > >> > > > > > > > > > > > state is reloaded, we need the new
>> > > > producer
>> > > > > ID
>> > > > > > > in
>> > > > > > > > >> the
>> > > > > > > > >> > > state
>> > > > > > > > >> > > > > so
>> > > > > > > > >> > > > > > > > there
>> > > > > > > > >> > > > > > > > > > > isn't
>> > > > > > > > >> > > > > > > > > > > > an invalid producer ID mapping.
>> > > > > > > > >> > > > > > > > > > > > The server downgrade cases are
>> > > considering
>> > > > > > > > >> > transactional
>> > > > > > > > >> > > > > > > > correctness
>> > > > > > > > >> > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > not regressing from previous
>> behavior
>> > --
>> > > > and
>> > > > > > are
>> > > > > > > > not
>> > > > > > > > >> > > > > concerned
>> > > > > > > > >> > > > > > > > about
>> > > > > > > > >> > > > > > > > > > > > supporting the safety from fencing
>> > > retries
>> > > > > (as
>> > > > > > > we
>> > > > > > > > >> have
>> > > > > > > > >> > > > > > downgraded
>> > > > > > > > >> > > > > > > > so
>> > > > > > > > >> > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > don't need to support). Perhaps
>> this
>> > is
>> > > a
>> > > > > > trade
>> > > > > > > > off,
>> > > > > > > > >> > but
>> > > > > > > > >> > > I
>> > > > > > > > >> > > > > > think
>> > > > > > > > >> > > > > > > it
>> > > > > > > > >> > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > right one.
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > (If the client downgrades, it will
>> > have
>> > > > > > > restarted
>> > > > > > > > >> and
>> > > > > > > > >> > it
>> > > > > > > > >> > > is
>> > > > > > > > >> > > > > ok
>> > > > > > > > >> > > > > > > for
>> > > > > > > > >> > > > > > > > it
>> > > > > > > > >> > > > > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > have a new producer ID too).
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > Justine
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > On Fri, Jan 12, 2024 at 11:42 AM
>> Jun
>> > Rao
>> > > > > > > > >> > > > > > > <jun@confluent.io.invalid
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > Thanks for the reply.
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > 101.4 "If the marker is written
>> by
>> > the
>> > > > new
>> > > > > > > > >> client, we
>> > > > > > > > >> > > can
>> > > > > > > > >> > > > > as
>> > > > > > > > >> > > > > > I
>> > > > > > > > >> > > > > > > > > > > mentioned
>> > > > > > > > >> > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > the last email guarantee that any
>> > > EndTxn
>> > > > > > > > requests
>> > > > > > > > >> > with
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > same
>> > > > > > > > >> > > > > > > > > epoch
>> > > > > > > > >> > > > > > > > > > > are
>> > > > > > > > >> > > > > > > > > > > > > from the same producer and the
>> same
>> > > > > > > transaction.
>> > > > > > > > >> Then
>> > > > > > > > >> > > we
>> > > > > > > > >> > > > > > don't
>> > > > > > > > >> > > > > > > > have
>> > > > > > > > >> > > > > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > > return a fenced error but can
>> handle
>> > > > > > > gracefully
>> > > > > > > > as
>> > > > > > > > >> > > > > described
>> > > > > > > > >> > > > > > in
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > KIP."
>> > > > > > > > >> > > > > > > > > > > > > When a delayed EndTnx request is
>> > > > > processed,
>> > > > > > > the
>> > > > > > > > >> txn
>> > > > > > > > >> > > state
>> > > > > > > > >> > > > > > could
>> > > > > > > > >> > > > > > > > be
>> > > > > > > > >> > > > > > > > > > > > ongoing
>> > > > > > > > >> > > > > > > > > > > > > for the next txn. I guess in this
>> > case
>> > > > we
>> > > > > > > still
>> > > > > > > > >> > return
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > fenced
>> > > > > > > > >> > > > > > > > > > error
>> > > > > > > > >> > > > > > > > > > > > for
>> > > > > > > > >> > > > > > > > > > > > > the delayed request?
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > 102. Sorry, my question was
>> > > inaccurate.
>> > > > > What
>> > > > > > > you
>> > > > > > > > >> > > > described
>> > > > > > > > >> > > > > is
>> > > > > > > > >> > > > > > > > > > accurate.
>> > > > > > > > >> > > > > > > > > > > > > "The downgrade compatibility I
>> > mention
>> > > > is
>> > > > > > that
>> > > > > > > > we
>> > > > > > > > >> > keep
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > same
>> > > > > > > > >> > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > > and epoch in the main
>> (non-tagged)
>> > > > fields
>> > > > > as
>> > > > > > > we
>> > > > > > > > >> did
>> > > > > > > > >> > > > before
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > code
>> > > > > > > > >> > > > > > > > > > on
>> > > > > > > > >> > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > server side." If we want to do
>> this,
>> > > it
>> > > > > > seems
>> > > > > > > > >> that we
>> > > > > > > > >> > > > > should
>> > > > > > > > >> > > > > > > use
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > current produce Id and max epoch
>> in
>> > > the
>> > > > > > > existing
>> > > > > > > > >> > > > producerId
>> > > > > > > > >> > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > producerEpoch fields for both the
>> > > > prepare
>> > > > > > and
>> > > > > > > > the
>> > > > > > > > >> > > > complete
>> > > > > > > > >> > > > > > > > marker,
>> > > > > > > > >> > > > > > > > > > > right?
>> > > > > > > > >> > > > > > > > > > > > > The downgrade can happen after
>> the
>> > > > > complete
>> > > > > > > > >> marker is
>> > > > > > > > >> > > > > > written.
>> > > > > > > > >> > > > > > > > With
>> > > > > > > > >> > > > > > > > > > > what
>> > > > > > > > >> > > > > > > > > > > > > you described, the downgraded
>> > > > coordinator
>> > > > > > will
>> > > > > > > > see
>> > > > > > > > >> > the
>> > > > > > > > >> > > > new
>> > > > > > > > >> > > > > > > > produce
>> > > > > > > > >> > > > > > > > > Id
>> > > > > > > > >> > > > > > > > > > > > > instead of the old one.
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > Jun
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > On Fri, Jan 12, 2024 at 10:44 AM
>> > > Justine
>> > > > > > > Olshan
>> > > > > > > > >> > > > > > > > > > > > > <jo...@confluent.io.invalid>
>> > wrote:
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > Hi Jun,
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > I can update the description.
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > I believe your second point is
>> > > > mentioned
>> > > > > > in
>> > > > > > > > the
>> > > > > > > > >> > KIP.
>> > > > > > > > >> > > I
>> > > > > > > > >> > > > > can
>> > > > > > > > >> > > > > > > add
>> > > > > > > > >> > > > > > > > > more
>> > > > > > > > >> > > > > > > > > > > > text
>> > > > > > > > >> > > > > > > > > > > > > on
>> > > > > > > > >> > > > > > > > > > > > > > this if it is helpful.
>> > > > > > > > >> > > > > > > > > > > > > > > The delayed message case can
>> > also
>> > > > > > violate
>> > > > > > > > EOS
>> > > > > > > > >> if
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > delayed
>> > > > > > > > >> > > > > > > > > > > message
>> > > > > > > > >> > > > > > > > > > > > > > comes in after the next
>> > > > > addPartitionsToTxn
>> > > > > > > > >> request
>> > > > > > > > >> > > > comes
>> > > > > > > > >> > > > > > in.
>> > > > > > > > >> > > > > > > > > > > > Effectively
>> > > > > > > > >> > > > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > may see a message from a
>> previous
>> > > > > > (aborted)
>> > > > > > > > >> > > transaction
>> > > > > > > > >> > > > > > > become
>> > > > > > > > >> > > > > > > > > part
>> > > > > > > > >> > > > > > > > > > > of
>> > > > > > > > >> > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > next transaction.
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > If the marker is written by the
>> > new
>> > > > > > client,
>> > > > > > > we
>> > > > > > > > >> can
>> > > > > > > > >> > > as I
>> > > > > > > > >> > > > > > > > mentioned
>> > > > > > > > >> > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > last email guarantee that any
>> > EndTxn
>> > > > > > > requests
>> > > > > > > > >> with
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > same
>> > > > > > > > >> > > > > > > > epoch
>> > > > > > > > >> > > > > > > > > > are
>> > > > > > > > >> > > > > > > > > > > > > from
>> > > > > > > > >> > > > > > > > > > > > > > the same producer and the same
>> > > > > > transaction.
>> > > > > > > > >> Then we
>> > > > > > > > >> > > > don't
>> > > > > > > > >> > > > > > > have
>> > > > > > > > >> > > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > return
>> > > > > > > > >> > > > > > > > > > > > > a
>> > > > > > > > >> > > > > > > > > > > > > > fenced error but can handle
>> > > gracefully
>> > > > > as
>> > > > > > > > >> described
>> > > > > > > > >> > > in
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > KIP.
>> > > > > > > > >> > > > > > > > > > > > > > I don't think a boolean is
>> useful
>> > > > since
>> > > > > it
>> > > > > > > is
>> > > > > > > > >> > > directly
>> > > > > > > > >> > > > > > > encoded
>> > > > > > > > >> > > > > > > > by
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > existence or lack of the tagged
>> > > field
>> > > > > > being
>> > > > > > > > >> > written.
>> > > > > > > > >> > > > > > > > > > > > > > In the prepare marker we will
>> have
>> > > the
>> > > > > > same
>> > > > > > > > >> > producer
>> > > > > > > > >> > > ID
>> > > > > > > > >> > > > > in
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > non-tagged
>> > > > > > > > >> > > > > > > > > > > > > > field. In the Complete state we
>> > may
>> > > > not.
>> > > > > > > > >> > > > > > > > > > > > > > I'm not sure why the ongoing
>> state
>> > > > > matters
>> > > > > > > for
>> > > > > > > > >> this
>> > > > > > > > >> > > > KIP.
>> > > > > > > > >> > > > > It
>> > > > > > > > >> > > > > > > > does
>> > > > > > > > >> > > > > > > > > > > matter
>> > > > > > > > >> > > > > > > > > > > > > for
>> > > > > > > > >> > > > > > > > > > > > > > KIP-939.
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > I'm not sure what you are
>> > referring
>> > > to
>> > > > > > about
>> > > > > > > > >> > writing
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > previous
>> > > > > > > > >> > > > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > ID in the prepare marker. This
>> is
>> > > not
>> > > > in
>> > > > > > the
>> > > > > > > > >> KIP.
>> > > > > > > > >> > > > > > > > > > > > > > In the overflow case, we write
>> the
>> > > > > > > > >> nextProducerId
>> > > > > > > > >> > in
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > prepare
>> > > > > > > > >> > > > > > > > > > > state.
>> > > > > > > > >> > > > > > > > > > > > > > This is so we know what we
>> > assigned
>> > > > when
>> > > > > > we
>> > > > > > > > >> reload
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > > > > transaction
>> > > > > > > > >> > > > > > > > > > > log.
>> > > > > > > > >> > > > > > > > > > > > > > Once we complete, we transition
>> > this
>> > > > ID
>> > > > > to
>> > > > > > > the
>> > > > > > > > >> main
>> > > > > > > > >> > > > > > > (non-tagged
>> > > > > > > > >> > > > > > > > > > > field)
>> > > > > > > > >> > > > > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > > have the previous producer ID
>> > field
>> > > > > filled
>> > > > > > > in.
>> > > > > > > > >> This
>> > > > > > > > >> > > is
>> > > > > > > > >> > > > so
>> > > > > > > > >> > > > > > we
>> > > > > > > > >> > > > > > > > can
>> > > > > > > > >> > > > > > > > > > > > identify
>> > > > > > > > >> > > > > > > > > > > > > > in a retry case the operation
>> > > > completed
>> > > > > > > > >> > successfully
>> > > > > > > > >> > > > and
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > > > don't
>> > > > > > > > >> > > > > > > > > > > fence
>> > > > > > > > >> > > > > > > > > > > > > our
>> > > > > > > > >> > > > > > > > > > > > > > producer. The downgrade
>> > > compatibility
>> > > > I
>> > > > > > > > mention
>> > > > > > > > >> is
>> > > > > > > > >> > > that
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > > keep
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > same
>> > > > > > > > >> > > > > > > > > > > > > > producer ID and epoch in the
>> main
>> > > > > > > (non-tagged)
>> > > > > > > > >> > fields
>> > > > > > > > >> > > > as
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > > did
>> > > > > > > > >> > > > > > > > > > > before
>> > > > > > > > >> > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > code on the server side. If the
>> > > server
>> > > > > > > > >> downgrades,
>> > > > > > > > >> > we
>> > > > > > > > >> > > > are
>> > > > > > > > >> > > > > > > still
>> > > > > > > > >> > > > > > > > > > > > > compatible.
>> > > > > > > > >> > > > > > > > > > > > > > This addresses both the prepare
>> > and
>> > > > > > complete
>> > > > > > > > >> state
>> > > > > > > > >> > > > > > > downgrades.
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > Justine
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > On Fri, Jan 12, 2024 at
>> 10:21 AM
>> > Jun
>> > > > Rao
>> > > > > > > > >> > > > > > > > > <jun@confluent.io.invalid
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > Thanks for the reply. Sorry
>> for
>> > > the
>> > > > > > > delay. I
>> > > > > > > > >> > have a
>> > > > > > > > >> > > > few
>> > > > > > > > >> > > > > > > more
>> > > > > > > > >> > > > > > > > > > > > comments.
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > 110. I think the motivation
>> > > section
>> > > > > > could
>> > > > > > > be
>> > > > > > > > >> > > > improved.
>> > > > > > > > >> > > > > > One
>> > > > > > > > >> > > > > > > of
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > motivations listed by the
>> KIP is
>> > > > "This
>> > > > > > can
>> > > > > > > > >> happen
>> > > > > > > > >> > > > when
>> > > > > > > > >> > > > > a
>> > > > > > > > >> > > > > > > > > message
>> > > > > > > > >> > > > > > > > > > > gets
>> > > > > > > > >> > > > > > > > > > > > > > stuck
>> > > > > > > > >> > > > > > > > > > > > > > > or delayed due to networking
>> > > issues
>> > > > > or a
>> > > > > > > > >> network
>> > > > > > > > >> > > > > > partition,
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > transaction
>> > > > > > > > >> > > > > > > > > > > > > > > aborts, and then the delayed
>> > > message
>> > > > > > > finally
>> > > > > > > > >> > comes
>> > > > > > > > >> > > > > in.".
>> > > > > > > > >> > > > > > > This
>> > > > > > > > >> > > > > > > > > > seems
>> > > > > > > > >> > > > > > > > > > > > not
>> > > > > > > > >> > > > > > > > > > > > > > > very accurate. Without
>> KIP-890,
>> > > > > > currently,
>> > > > > > > > if
>> > > > > > > > >> the
>> > > > > > > > >> > > > > > > coordinator
>> > > > > > > > >> > > > > > > > > > times
>> > > > > > > > >> > > > > > > > > > > > out
>> > > > > > > > >> > > > > > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > > > aborts an ongoing
>> transaction,
>> > it
>> > > > > > already
>> > > > > > > > >> bumps
>> > > > > > > > >> > up
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > epoch
>> > > > > > > > >> > > > > > > > in
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > marker,
>> > > > > > > > >> > > > > > > > > > > > > > > which prevents the delayed
>> > produce
>> > > > > > message
>> > > > > > > > >> from
>> > > > > > > > >> > > being
>> > > > > > > > >> > > > > > added
>> > > > > > > > >> > > > > > > > to
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > user
>> > > > > > > > >> > > > > > > > > > > > > > > partition. What can cause a
>> > > hanging
>> > > > > > > > >> transaction
>> > > > > > > > >> > is
>> > > > > > > > >> > > > that
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > > completes (either aborts or
>> > > > commits) a
>> > > > > > > > >> > transaction
>> > > > > > > > >> > > > > before
>> > > > > > > > >> > > > > > > > > > > receiving a
>> > > > > > > > >> > > > > > > > > > > > > > > successful ack on messages
>> > > published
>> > > > > in
>> > > > > > > the
>> > > > > > > > >> same
>> > > > > > > > >> > > txn.
>> > > > > > > > >> > > > > In
>> > > > > > > > >> > > > > > > this
>> > > > > > > > >> > > > > > > > > > case,
>> > > > > > > > >> > > > > > > > > > > > > it's
>> > > > > > > > >> > > > > > > > > > > > > > > possible for the delayed
>> message
>> > > to
>> > > > be
>> > > > > > > > >> appended
>> > > > > > > > >> > to
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > > partition
>> > > > > > > > >> > > > > > > > > > > > after
>> > > > > > > > >> > > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > marker, causing a
>> transaction to
>> > > > hang.
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > A similar issue (not
>> mentioned
>> > in
>> > > > the
>> > > > > > > > >> motivation)
>> > > > > > > > >> > > > could
>> > > > > > > > >> > > > > > > > happen
>> > > > > > > > >> > > > > > > > > on
>> > > > > > > > >> > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > marker in the coordinator's
>> log.
>> > > For
>> > > > > > > > example,
>> > > > > > > > >> > it's
>> > > > > > > > >> > > > > > possible
>> > > > > > > > >> > > > > > > > for
>> > > > > > > > >> > > > > > > > > > an
>> > > > > > > > >> > > > > > > > > > > > > > > EndTxnRequest to be delayed
>> on
>> > the
>> > > > > > > > >> coordinator.
>> > > > > > > > >> > By
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > time
>> > > > > > > > >> > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > delayed
>> > > > > > > > >> > > > > > > > > > > > > > > EndTxnRequest is processed,
>> it's
>> > > > > > possible
>> > > > > > > > that
>> > > > > > > > >> > the
>> > > > > > > > >> > > > > > previous
>> > > > > > > > >> > > > > > > > txn
>> > > > > > > > >> > > > > > > > > > has
>> > > > > > > > >> > > > > > > > > > > > > > already
>> > > > > > > > >> > > > > > > > > > > > > > > completed and a new txn has
>> > > started.
>> > > > > > > > >> Currently,
>> > > > > > > > >> > > since
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > epoch
>> > > > > > > > >> > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > not
>> > > > > > > > >> > > > > > > > > > > > > > > bumped on every txn, the
>> delayed
>> > > > > > > > EndTxnRequest
>> > > > > > > > >> > will
>> > > > > > > > >> > > > add
>> > > > > > > > >> > > > > > an
>> > > > > > > > >> > > > > > > > > > > unexpected
>> > > > > > > > >> > > > > > > > > > > > > > > prepare marker (and
>> eventually a
>> > > > > > complete
>> > > > > > > > >> marker)
>> > > > > > > > >> > > to
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > > ongoing
>> > > > > > > > >> > > > > > > > > > > txn.
>> > > > > > > > >> > > > > > > > > > > > > > This
>> > > > > > > > >> > > > > > > > > > > > > > > won't cause the transaction
>> to
>> > > hang,
>> > > > > but
>> > > > > > > it
>> > > > > > > > >> will
>> > > > > > > > >> > > > break
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > EoS
>> > > > > > > > >> > > > > > > > > > > > > semantic.
>> > > > > > > > >> > > > > > > > > > > > > > > The proposal in this KIP will
>> > > > address
>> > > > > > this
>> > > > > > > > >> issue
>> > > > > > > > >> > > too.
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > 101. "However, I was writing
>> it
>> > so
>> > > > > that
>> > > > > > we
>> > > > > > > > can
>> > > > > > > > >> > > > > > distinguish
>> > > > > > > > >> > > > > > > > > > between
>> > > > > > > > >> > > > > > > > > > > > > > > old clients where we don't
>> have
>> > > the
>> > > > > > > ability
>> > > > > > > > do
>> > > > > > > > >> > this
>> > > > > > > > >> > > > > > > operation
>> > > > > > > > >> > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > new
>> > > > > > > > >> > > > > > > > > > > > > > > clients that can. (Old
>> clients
>> > > don't
>> > > > > > bump
>> > > > > > > > the
>> > > > > > > > >> > epoch
>> > > > > > > > >> > > > on
>> > > > > > > > >> > > > > > > > commit,
>> > > > > > > > >> > > > > > > > > so
>> > > > > > > > >> > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > can't
>> > > > > > > > >> > > > > > > > > > > > > > > say for sure the write
>> belongs
>> > to
>> > > > the
>> > > > > > > given
>> > > > > > > > >> > > > > > transaction)."
>> > > > > > > > >> > > > > > > > > > > > > > > 101.1 I am wondering why we
>> need
>> > > to
>> > > > > > > > >> distinguish
>> > > > > > > > >> > > > whether
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > > > marker
>> > > > > > > > >> > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > written by the old and the
>> new
>> > > > client.
>> > > > > > > Could
>> > > > > > > > >> you
>> > > > > > > > >> > > > > describe
>> > > > > > > > >> > > > > > > > what
>> > > > > > > > >> > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > do
>> > > > > > > > >> > > > > > > > > > > > > > > differently if we know the
>> > marker
>> > > is
>> > > > > > > written
>> > > > > > > > >> by
>> > > > > > > > >> > the
>> > > > > > > > >> > > > new
>> > > > > > > > >> > > > > > > > client?
>> > > > > > > > >> > > > > > > > > > > > > > > 101.2 If we do need a way to
>> > > > > distinguish
>> > > > > > > > >> whether
>> > > > > > > > >> > > the
>> > > > > > > > >> > > > > > marker
>> > > > > > > > >> > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > written
>> > > > > > > > >> > > > > > > > > > > > > by
>> > > > > > > > >> > > > > > > > > > > > > > > the old and the new client.
>> > Would
>> > > it
>> > > > > be
>> > > > > > > > >> simpler
>> > > > > > > > >> > to
>> > > > > > > > >> > > > just
>> > > > > > > > >> > > > > > > > > > introduce a
>> > > > > > > > >> > > > > > > > > > > > > > boolean
>> > > > > > > > >> > > > > > > > > > > > > > > field instead of indirectly
>> > > through
>> > > > > the
>> > > > > > > > >> previous
>> > > > > > > > >> > > > > produce
>> > > > > > > > >> > > > > > ID
>> > > > > > > > >> > > > > > > > > > field?
>> > > > > > > > >> > > > > > > > > > > > > > > 101.3 It's not clear to me
>> why
>> > we
>> > > > only
>> > > > > > add
>> > > > > > > > the
>> > > > > > > > >> > > > previous
>> > > > > > > > >> > > > > > > > produce
>> > > > > > > > >> > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > > field
>> > > > > > > > >> > > > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > > > the complete marker, but not
>> in
>> > > the
>> > > > > > > prepare
>> > > > > > > > >> > marker.
>> > > > > > > > >> > > > If
>> > > > > > > > >> > > > > we
>> > > > > > > > >> > > > > > > > want
>> > > > > > > > >> > > > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > know
>> > > > > > > > >> > > > > > > > > > > > > > > whether a marker is written
>> by
>> > the
>> > > > new
>> > > > > > > > client
>> > > > > > > > >> or
>> > > > > > > > >> > > not,
>> > > > > > > > >> > > > > it
>> > > > > > > > >> > > > > > > > seems
>> > > > > > > > >> > > > > > > > > > that
>> > > > > > > > >> > > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > want
>> > > > > > > > >> > > > > > > > > > > > > > > to do this consistently for
>> all
>> > > > > markers.
>> > > > > > > > >> > > > > > > > > > > > > > > 101.4 What about the
>> > > > > TransactionLogValue
>> > > > > > > > >> record
>> > > > > > > > >> > > > > > > representing
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > ongoing
>> > > > > > > > >> > > > > > > > > > > > > > > state? Should we also
>> > distinguish
>> > > > > > whether
>> > > > > > > > it's
>> > > > > > > > >> > > > written
>> > > > > > > > >> > > > > by
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > old
>> > > > > > > > >> > > > > > > > > > > or
>> > > > > > > > >> > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > new client?
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > 102. In the overflow case,
>> it's
>> > > > still
>> > > > > > not
>> > > > > > > > >> clear
>> > > > > > > > >> > to
>> > > > > > > > >> > > me
>> > > > > > > > >> > > > > why
>> > > > > > > > >> > > > > > > we
>> > > > > > > > >> > > > > > > > > > write
>> > > > > > > > >> > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > previous produce Id in the
>> > prepare
>> > > > > > marker
>> > > > > > > > >> while
>> > > > > > > > >> > > > writing
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > > next
>> > > > > > > > >> > > > > > > > > > > > > produce
>> > > > > > > > >> > > > > > > > > > > > > > Id
>> > > > > > > > >> > > > > > > > > > > > > > > in the complete marker. You
>> > > > mentioned
>> > > > > > that
>> > > > > > > > >> it's
>> > > > > > > > >> > for
>> > > > > > > > >> > > > > > > > > downgrading.
>> > > > > > > > >> > > > > > > > > > > > > However,
>> > > > > > > > >> > > > > > > > > > > > > > > we could downgrade with
>> either
>> > the
>> > > > > > prepare
>> > > > > > > > >> marker
>> > > > > > > > >> > > or
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > > complete
>> > > > > > > > >> > > > > > > > > > > > > marker.
>> > > > > > > > >> > > > > > > > > > > > > > > In either case, the
>> downgraded
>> > > > > > coordinator
>> > > > > > > > >> should
>> > > > > > > > >> > > see
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > same
>> > > > > > > > >> > > > > > > > > > > > produce
>> > > > > > > > >> > > > > > > > > > > > > id
>> > > > > > > > >> > > > > > > > > > > > > > > (probably the previous
>> produce
>> > > Id),
>> > > > > > right?
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > Jun
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > On Wed, Dec 20, 2023 at
>> 6:00 PM
>> > > > > Justine
>> > > > > > > > Olshan
>> > > > > > > > >> > > > > > > > > > > > > > > <jolshan@confluent.io.invalid
>> >
>> > > > > > > > >> > > > > > > > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > Hey Jun,
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > Thanks for taking a look at
>> > the
>> > > > KIP
>> > > > > > > again.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > 100. For the epoch overflow
>> > > case,
>> > > > > only
>> > > > > > > the
>> > > > > > > > >> > marker
>> > > > > > > > >> > > > > will
>> > > > > > > > >> > > > > > > have
>> > > > > > > > >> > > > > > > > > max
>> > > > > > > > >> > > > > > > > > > > > > epoch.
>> > > > > > > > >> > > > > > > > > > > > > > > This
>> > > > > > > > >> > > > > > > > > > > > > > > > keeps the behavior of the
>> rest
>> > > of
>> > > > > the
>> > > > > > > > >> markers
>> > > > > > > > >> > > where
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > last
>> > > > > > > > >> > > > > > > > > > > marker
>> > > > > > > > >> > > > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > epoch of the transaction
>> > > records +
>> > > > > 1.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > 101. You are correct that
>> we
>> > > don't
>> > > > > > need
>> > > > > > > to
>> > > > > > > > >> > write
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > > since
>> > > > > > > > >> > > > > > > > > > > > > > it
>> > > > > > > > >> > > > > > > > > > > > > > > > is the same. However, I was
>> > > > writing
>> > > > > it
>> > > > > > > so
>> > > > > > > > >> that
>> > > > > > > > >> > we
>> > > > > > > > >> > > > can
>> > > > > > > > >> > > > > > > > > > distinguish
>> > > > > > > > >> > > > > > > > > > > > > > between
>> > > > > > > > >> > > > > > > > > > > > > > > > old clients where we don't
>> > have
>> > > > the
>> > > > > > > > ability
>> > > > > > > > >> do
>> > > > > > > > >> > > this
>> > > > > > > > >> > > > > > > > operation
>> > > > > > > > >> > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > new
>> > > > > > > > >> > > > > > > > > > > > > > > > clients that can. (Old
>> clients
>> > > > don't
>> > > > > > > bump
>> > > > > > > > >> the
>> > > > > > > > >> > > epoch
>> > > > > > > > >> > > > > on
>> > > > > > > > >> > > > > > > > > commit,
>> > > > > > > > >> > > > > > > > > > so
>> > > > > > > > >> > > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > > can't
>> > > > > > > > >> > > > > > > > > > > > > > > > say for sure the write
>> belongs
>> > > to
>> > > > > the
>> > > > > > > > given
>> > > > > > > > >> > > > > > transaction).
>> > > > > > > > >> > > > > > > > If
>> > > > > > > > >> > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > receive
>> > > > > > > > >> > > > > > > > > > > > > > > an
>> > > > > > > > >> > > > > > > > > > > > > > > > EndTxn request from a new
>> > > client,
>> > > > we
>> > > > > > > will
>> > > > > > > > >> fill
>> > > > > > > > >> > > this
>> > > > > > > > >> > > > > > > field.
>> > > > > > > > >> > > > > > > > We
>> > > > > > > > >> > > > > > > > > > can
>> > > > > > > > >> > > > > > > > > > > > > > > guarantee
>> > > > > > > > >> > > > > > > > > > > > > > > > that any EndTxn requests
>> with
>> > > the
>> > > > > same
>> > > > > > > > epoch
>> > > > > > > > >> > are
>> > > > > > > > >> > > > from
>> > > > > > > > >> > > > > > the
>> > > > > > > > >> > > > > > > > > same
>> > > > > > > > >> > > > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > > > > the same transaction.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > 102. In prepare phase, we
>> have
>> > > the
>> > > > > > same
>> > > > > > > > >> > producer
>> > > > > > > > >> > > ID
>> > > > > > > > >> > > > > and
>> > > > > > > > >> > > > > > > > epoch
>> > > > > > > > >> > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > always
>> > > > > > > > >> > > > > > > > > > > > > > > > had. It is the producer ID
>> and
>> > > > epoch
>> > > > > > > that
>> > > > > > > > >> are
>> > > > > > > > >> > on
>> > > > > > > > >> > > > the
>> > > > > > > > >> > > > > > > > marker.
>> > > > > > > > >> > > > > > > > > In
>> > > > > > > > >> > > > > > > > > > > > > commit
>> > > > > > > > >> > > > > > > > > > > > > > > > phase, we stay the same
>> unless
>> > > it
>> > > > is
>> > > > > > the
>> > > > > > > > >> > overflow
>> > > > > > > > >> > > > > case.
>> > > > > > > > >> > > > > > > In
>> > > > > > > > >> > > > > > > > > that
>> > > > > > > > >> > > > > > > > > > > > case,
>> > > > > > > > >> > > > > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > > > set the producer ID to the
>> new
>> > > one
>> > > > > we
>> > > > > > > > >> generated
>> > > > > > > > >> > > and
>> > > > > > > > >> > > > > > epoch
>> > > > > > > > >> > > > > > > > to
>> > > > > > > > >> > > > > > > > > 0
>> > > > > > > > >> > > > > > > > > > > > after
>> > > > > > > > >> > > > > > > > > > > > > > > > complete. This is for
>> > downgrade
>> > > > > > > > >> compatibility.
>> > > > > > > > >> > > The
>> > > > > > > > >> > > > > > tagged
>> > > > > > > > >> > > > > > > > > > fields
>> > > > > > > > >> > > > > > > > > > > > are
>> > > > > > > > >> > > > > > > > > > > > > > just
>> > > > > > > > >> > > > > > > > > > > > > > > > safety guards for retries
>> and
>> > > > > > failovers.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > In prepare phase for epoch
>> > > > overflow
>> > > > > > case
>> > > > > > > > >> only
>> > > > > > > > >> > we
>> > > > > > > > >> > > > > store
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > next
>> > > > > > > > >> > > > > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > > > ID. This is for the case
>> where
>> > > we
>> > > > > > reload
>> > > > > > > > the
>> > > > > > > > >> > > > > > transaction
>> > > > > > > > >> > > > > > > > > > > > coordinator
>> > > > > > > > >> > > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > > > > prepare state. Once the
>> > > > transaction
>> > > > > is
>> > > > > > > > >> > committed,
>> > > > > > > > >> > > > we
>> > > > > > > > >> > > > > > can
>> > > > > > > > >> > > > > > > > use
>> > > > > > > > >> > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > > > ID the client already is
>> > using.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > In commit phase, we store
>> the
>> > > > > previous
>> > > > > > > > >> producer
>> > > > > > > > >> > > ID
>> > > > > > > > >> > > > in
>> > > > > > > > >> > > > > > > case
>> > > > > > > > >> > > > > > > > of
>> > > > > > > > >> > > > > > > > > > > > > retries.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > I think it is easier to
>> think
>> > of
>> > > > it
>> > > > > as
>> > > > > > > > just
>> > > > > > > > >> how
>> > > > > > > > >> > > we
>> > > > > > > > >> > > > > were
>> > > > > > > > >> > > > > > > > > storing
>> > > > > > > > >> > > > > > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > > > > > and epoch before, with some
>> > > extra
>> > > > > > > > bookeeping
>> > > > > > > > >> > and
>> > > > > > > > >> > > > edge
>> > > > > > > > >> > > > > > > case
>> > > > > > > > >> > > > > > > > > > > handling
>> > > > > > > > >> > > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > tagged fields. We have to
>> do
>> > it
>> > > > this
>> > > > > > way
>> > > > > > > > for
>> > > > > > > > >> > > > > > > compatibility
>> > > > > > > > >> > > > > > > > > with
>> > > > > > > > >> > > > > > > > > > > > > > > downgrades.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > 103. Next producer ID is
>> for
>> > > > prepare
>> > > > > > > > status
>> > > > > > > > >> and
>> > > > > > > > >> > > > > > previous
>> > > > > > > > >> > > > > > > > > > producer
>> > > > > > > > >> > > > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > for
>> > > > > > > > >> > > > > > > > > > > > > > > > after complete. The reason
>> why
>> > > we
>> > > > > need
>> > > > > > > two
>> > > > > > > > >> > > separate
>> > > > > > > > >> > > > > > > > (tagged)
>> > > > > > > > >> > > > > > > > > > > fields
>> > > > > > > > >> > > > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > for
>> > > > > > > > >> > > > > > > > > > > > > > > > backwards compatibility. We
>> > need
>> > > > to
>> > > > > > keep
>> > > > > > > > the
>> > > > > > > > >> > same
>> > > > > > > > >> > > > > > > semantics
>> > > > > > > > >> > > > > > > > > for
>> > > > > > > > >> > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > non-tagged field in case we
>> > > > > downgrade.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > 104. We set the fields as
>> we
>> > do
>> > > in
>> > > > > the
>> > > > > > > > >> > > > transactional
>> > > > > > > > >> > > > > > > state
>> > > > > > > > >> > > > > > > > > (as
>> > > > > > > > >> > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > need
>> > > > > > > > >> > > > > > > > > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > > > > > do this for compatibility
>> --
>> > if
>> > > we
>> > > > > > > > >> downgrade,
>> > > > > > > > >> > we
>> > > > > > > > >> > > > will
>> > > > > > > > >> > > > > > > only
>> > > > > > > > >> > > > > > > > > have
>> > > > > > > > >> > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > non-tagged fields) It will
>> be
>> > > the
>> > > > > old
>> > > > > > > > >> producer
>> > > > > > > > >> > ID
>> > > > > > > > >> > > > and
>> > > > > > > > >> > > > > > max
>> > > > > > > > >> > > > > > > > > > epoch.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > Hope this helps. Let me
>> know
>> > if
>> > > > you
>> > > > > > have
>> > > > > > > > >> > further
>> > > > > > > > >> > > > > > > questions.
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > Justine
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > On Wed, Dec 20, 2023 at
>> > 3:33 PM
>> > > > Jun
>> > > > > > Rao
>> > > > > > > > >> > > > > > > > > > <jun@confluent.io.invalid
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > wrote:
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > Hi, Justine,
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > It seems that you have
>> made
>> > > some
>> > > > > > > changes
>> > > > > > > > >> to
>> > > > > > > > >> > > > KIP-890
>> > > > > > > > >> > > > > > > since
>> > > > > > > > >> > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > vote.
>> > > > > > > > >> > > > > > > > > > > > > > In
>> > > > > > > > >> > > > > > > > > > > > > > > > > particular, we are
>> changing
>> > > the
>> > > > > > format
>> > > > > > > > of
>> > > > > > > > >> > > > > > > > > > TransactionLogValue.
>> > > > > > > > >> > > > > > > > > > > A
>> > > > > > > > >> > > > > > > > > > > > > few
>> > > > > > > > >> > > > > > > > > > > > > > > > > comments related to that.
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > 100. Just to be clear.
>> The
>> > > > > overflow
>> > > > > > > case
>> > > > > > > > >> > (i.e.
>> > > > > > > > >> > > > > when a
>> > > > > > > > >> > > > > > > new
>> > > > > > > > >> > > > > > > > > > > > > producerId
>> > > > > > > > >> > > > > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > > > generated) is when the
>> > current
>> > > > > epoch
>> > > > > > > > >> equals
>> > > > > > > > >> > to
>> > > > > > > > >> > > > max
>> > > > > > > > >> > > > > -
>> > > > > > > > >> > > > > > 1
>> > > > > > > > >> > > > > > > > and
>> > > > > > > > >> > > > > > > > > > not
>> > > > > > > > >> > > > > > > > > > > > max?
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > 101. For the "not epoch
>> > > > overflow"
>> > > > > > > case,
>> > > > > > > > we
>> > > > > > > > >> > > write
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > > previous
>> > > > > > > > >> > > > > > > > > > > ID
>> > > > > > > > >> > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > > tagged field in the
>> complete
>> > > > > phase.
>> > > > > > Do
>> > > > > > > > we
>> > > > > > > > >> > need
>> > > > > > > > >> > > to
>> > > > > > > > >> > > > > do
>> > > > > > > > >> > > > > > > that
>> > > > > > > > >> > > > > > > > > > since
>> > > > > > > > >> > > > > > > > > > > > > > produce
>> > > > > > > > >> > > > > > > > > > > > > > > > id
>> > > > > > > > >> > > > > > > > > > > > > > > > > doesn't change in this
>> case?
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > 102. It seems that the
>> > meaning
>> > > > for
>> > > > > > the
>> > > > > > > > >> > > > > > > > > > ProducerId/ProducerEpoch
>> > > > > > > > >> > > > > > > > > > > > > > fields
>> > > > > > > > >> > > > > > > > > > > > > > > in
>> > > > > > > > >> > > > > > > > > > > > > > > > > TransactionLogValue
>> changes
>> > > > > > depending
>> > > > > > > on
>> > > > > > > > >> the
>> > > > > > > > >> > > > > > > > > > TransactionStatus.
>> > > > > > > > >> > > > > > > > > > > > > When
>> > > > > > > > >> > > > > > > > > > > > > > > > > the TransactionStatus is
>> > > > ongoing,
>> > > > > > they
>> > > > > > > > >> > > represent
>> > > > > > > > >> > > > > the
>> > > > > > > > >> > > > > > > > > current
>> > > > > > > > >> > > > > > > > > > > > > > ProducerId
>> > > > > > > > >> > > > > > > > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > > > > > the current
>> ProducerEpoch.
>> > > When
>> > > > > the
>> > > > > > > > >> > > > > TransactionStatus
>> > > > > > > > >> > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> PrepareCommit/PrepareAbort,
>> > > they
>> > > > > > > > represent
>> > > > > > > > >> > the
>> > > > > > > > >> > > > > > current
>> > > > > > > > >> > > > > > > > > > > ProducerId
>> > > > > > > > >> > > > > > > > > > > > > and
>> > > > > > > > >> > > > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > > next ProducerEpoch. When
>> the
>> > > > > > > > >> > TransactionStatus
>> > > > > > > > >> > > is
>> > > > > > > > >> > > > > > > > > > Commit/Abort,
>> > > > > > > > >> > > > > > > > > > > > > they
>> > > > > > > > >> > > > > > > > > > > > > > > > > further depend on whether
>> > the
>> > > > > epoch
>> > > > > > > > >> overflows
>> > > > > > > > >> > > or
>> > > > > > > > >> > > > > not.
>> > > > > > > > >> > > > > > > If
>> > > > > > > > >> > > > > > > > > > there
>> > > > > > > > >> > > > > > > > > > > is
>> > > > > > > > >> > > > > > > > > > > > > no
>> > > > > > > > >> > > > > > > > > > > > > > > > > overflow, they represent
>> > the
>> > > > > > current
>> > > > > > > > >> > > ProducerId
>> > > > > > > > >> > > > > and
>> > > > > > > > >> > > > > > > the
>> > > > > > > > >> > > > > > > > > next
>> > > > > > > > >> > > > > > > > > > > > > > > > ProducerEpoch
>> > > > > > > > >> > > > > > > > > > > > > > > > > (max). Otherwise, they
>> > > represent
>> > > > > the
>> > > > > > > > newly
>> > > > > > > > >> > > > > generated
>> > > > > > > > >> > > > > > > > > > ProducerId
>> > > > > > > > >> > > > > > > > > > > > > and a
>> > > > > > > > >> > > > > > > > > > > > > > > > > ProducerEpoch of 0. Is
>> that
>> > > > right?
>> > > > > > > This
>> > > > > > > > >> seems
>> > > > > > > > >> > > not
>> > > > > > > > >> > > > > > easy
>> > > > > > > > >> > > > > > > to
>> > > > > > > > >> > > > > > > > > > > > > understand.
>> > > > > > > > >> > > > > > > > > > > > > > > > Could
>> > > > > > > > >> > > > > > > > > > > > > > > > > we provide some examples
>> > like
>> > > > what
>> > > > > > > Artem
>> > > > > > > > >> has
>> > > > > > > > >> > > done
>> > > > > > > > >> > > > > in
>> > > > > > > > >> > > > > > > > > KIP-939?
>> > > > > > > > >> > > > > > > > > > > > Have
>> > > > > > > > >> > > > > > > > > > > > > we
>> > > > > > > > >> > > > > > > > > > > > > > > > > considered a simpler
>> design
>> > > > where
>> > > > > > > > >> > > > > > > > ProducerId/ProducerEpoch
>> > > > > > > > >> > > > > > > > > > > always
>> > > > > > > > >> > > > > > > > > > > > > > > > represent
>> > > > > > > > >> > > > > > > > > > > > > > > > > the same value (e.g. for
>> the
>> > > > > current
>> > > > > > > > >> > > transaction)
>> > > > > > > > >> > > > > > > > > independent
>> > > > > > > > >> > > > > > > > > > > of
>> > > > > > > > >> > > > > > > > > > > > > the
>> > > > > > > > >> > > > > > > > > > > > > > > > > TransactionStatus and
>> epoch
>> > > > > > overflow?
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > 103. It's not clear to me
>> > why
>> > > we
>> > > > > > need
>> > > > > > > 3
>> > > > > > > > >> > fields:
>> > > > > > > > >> > > > > > > > ProducerId,
>> > > > > > > > >> > > > > > > > > > > > > > > > PrevProducerId,
>> > > > > > > > >> > > > > > > > > > > > > > > > > NextProducerId. Could we
>> > just
>> > > > have
>> > > > > > > > >> ProducerId
>> > > > > > > > >> > > and
>> > > > > > > > >> > > > > > > > > > > NextProducerId?
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > 104. For
>> > > WriteTxnMarkerRequests,
>> > > > > if
>> > > > > > > the
>> > > > > > > > >> > > producer
>> > > > > > > > >> > > > > > epoch
>> > > > > > > > >> > > > > > > > > > > overflows,
>> > > > > > > > >> > > > > > > > > > > > > > what
>> > > > > > > > >> > > > > > > > > > > > > > > do
>> > > > > > > > >> > > > > > > > > > > > > > > > > we set the producerId and
>> > the
>> > > > > > > > >> producerEpoch?
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > > > Jun
>> > > > > > > > >> > > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > > >
>> > > > > > > > >> > > > > > > > > >
>> > > > > > > > >> > > > > > > > >
>> > > > > > > > >> > > > > > > >
>> > > > > > > > >> > > > > > >
>> > > > > > > > >> > > > > >
>> > > > > > > > >> > > > >
>> > > > > > > > >> > > >
>> > > > > > > > >> > >
>> > > > > > > > >> >
>> > > > > > > > >>
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>

Re: [DISCUSS] KIP-890 Server Side Defense

Posted by Justine Olshan <jo...@confluent.io.INVALID>.
Hi there -- another update!

When looking into the implementation for the safe epoch bumps I realized
that we are already populating previousProducerID in memory as part of
KIP-360.
If we are to start using flexible fields, it is better to always use this
information and have an explicit (tagged) field to indicate whether the
client supports KIP-890 part 2.

I've included the extra field and how it is set in the KIP. I've also
updated the KIP to explain that we will be setting the tagged fields when
they are available for all transitions.

Finally, I added clearer text about the transaction protocol versions
included with this KIP. 1 for flexible transaction state records and 2 for
KIP-890 part 2 enablement.

Justine

On Mon, Mar 18, 2024 at 6:39 PM Justine Olshan <jo...@confluent.io> wrote:

> Hey there -- small update to the KIP,
>
> The KIP mentioned introducing ABORTABLE_ERROR and bumping TxnOffsetCommit
> and Produce requests. I've changed the name in the KIP to
> ABORTABLE_TRANSACTION and the corresponding exception
> AbortableTransactionException to match the pattern we had for other errors.
> I also mentioned bumping all 6 transactional APIs so we can future
> proof/support the error on the client going forward. If a future change
> wants to have an error scenario that requires us to abort the transaction,
> we can rely on the 3.8+ clients to support it. We ran into issues finding
> good/generic error codes that older clients could support while working on
> this KIP, so this should help in the future.
>
> The features discussion is still ongoing in KIP-1022. Will update again
> here when that concludes.
>
> Justine
>
> On Tue, Feb 6, 2024 at 8:39 AM Justine Olshan <jo...@confluent.io>
> wrote:
>
>> I don't think AddPartitions is a good example since we currenly don't
>> gate the version on TV or MV. (We only set a different flag depending on
>> the TV)
>>
>> Even if we did want to gate it on TV, I think the idea is to move away
>> from MV gating inter broker protocols. Ideally we can get to a state where
>> MV is just used for metadata changes.
>>
>> I think some of this discussion might fit more with the feature version
>> KIP, so I can try to open that up soon. Until we settle that, some of the
>> work in KIP-890 is blocked.
>>
>> Justine
>>
>> On Mon, Feb 5, 2024 at 5:38 PM Jun Rao <ju...@confluent.io.invalid> wrote:
>>
>>> Hi, Justine,
>>>
>>> Thanks for the reply.
>>>
>>> Since AddPartitions is an inter broker request, will its version be gated
>>> only by TV or other features like MV too? For example, if we need to
>>> change
>>> the protocol for AddPartitions for reasons other than txn verification in
>>> the future, will the new version be gated by a new MV? If so, does
>>> downgrading a TV imply potential downgrade of MV too?
>>>
>>> Jun
>>>
>>>
>>>
>>> On Mon, Feb 5, 2024 at 5:07 PM Justine Olshan
>>> <jo...@confluent.io.invalid>
>>> wrote:
>>>
>>> > One TV gates the flexible feature version (no rpcs involved, only the
>>> > transactional records that should only be gated by TV)
>>> > Another TV gates the ability to turn on kip-890 part 2. This would
>>> gate the
>>> > version of Produce and EndTxn (likely only used by transactions), and
>>> > specifies a flag in AddPartitionsToTxn though the version is already
>>> used
>>> > without TV.
>>> >
>>> > I think the only concern is the Produce request and we could consider
>>> work
>>> > arounds similar to the AddPartitionsToTxn call.
>>> >
>>> > Justine
>>> >
>>> > On Mon, Feb 5, 2024 at 4:56 PM Jun Rao <ju...@confluent.io.invalid>
>>> wrote:
>>> >
>>> > > Hi, Justine,
>>> > >
>>> > > Which PRC/record protocols will TV guard? Going forward, will those
>>> > > PRC/record protocols only be guarded by TV and not by other features
>>> like
>>> > > MV?
>>> > >
>>> > > Thanks,
>>> > >
>>> > > Jun
>>> > >
>>> > > On Mon, Feb 5, 2024 at 2:41 PM Justine Olshan
>>> > <jolshan@confluent.io.invalid
>>> > > >
>>> > > wrote:
>>> > >
>>> > > > Hi Jun,
>>> > > >
>>> > > > Sorry I think I misunderstood your question or answered
>>> incorrectly.
>>> > The
>>> > > TV
>>> > > > version should ideally be fully independent from MV.
>>> > > > At least for the changes I proposed, TV should not affect MV and MV
>>> > > should
>>> > > > not affect TV/
>>> > > >
>>> > > > I think if we downgrade TV, only that feature should downgrade.
>>> > Likewise
>>> > > > the same with MV. The finalizedFeatures should just reflect the
>>> feature
>>> > > > downgrade we made.
>>> > > >
>>> > > > I also plan to write a new KIP for managing the disk format and
>>> upgrade
>>> > > > tool as we will need new flags to support these features. That
>>> should
>>> > > help
>>> > > > clarify some things.
>>> > > >
>>> > > > Justine
>>> > > >
>>> > > > On Mon, Feb 5, 2024 at 11:03 AM Jun Rao <ju...@confluent.io.invalid>
>>> > > wrote:
>>> > > >
>>> > > > > Hi, Justine,
>>> > > > >
>>> > > > > Thanks for the reply.
>>> > > > >
>>> > > > > So, if we downgrade TV, we could implicitly downgrade another
>>> feature
>>> > > > (say
>>> > > > > MV) that has dependency (e.g. RPC). What would we return for
>>> > > > > FinalizedFeatures for MV in ApiVersionsResponse in that case?
>>> > > > >
>>> > > > > Thanks,
>>> > > > >
>>> > > > > Jun
>>> > > > >
>>> > > > > On Fri, Feb 2, 2024 at 1:06 PM Justine Olshan
>>> > > > <jolshan@confluent.io.invalid
>>> > > > > >
>>> > > > > wrote:
>>> > > > >
>>> > > > > > Hey Jun,
>>> > > > > >
>>> > > > > > Yes, the idea is that if we downgrade TV (transaction version)
>>> we
>>> > > will
>>> > > > > stop
>>> > > > > > using the add partitions to txn optimization and stop writing
>>> the
>>> > > > > flexible
>>> > > > > > feature version of the log.
>>> > > > > > In the compatibility section I included some explanations on
>>> how
>>> > this
>>> > > > is
>>> > > > > > done.
>>> > > > > >
>>> > > > > > Thanks,
>>> > > > > > Justine
>>> > > > > >
>>> > > > > > On Fri, Feb 2, 2024 at 11:12 AM Jun Rao
>>> <ju...@confluent.io.invalid>
>>> > > > > wrote:
>>> > > > > >
>>> > > > > > > Hi, Justine,
>>> > > > > > >
>>> > > > > > > Thanks for the update.
>>> > > > > > >
>>> > > > > > > If we ever downgrade the transaction feature, any feature
>>> > depending
>>> > > > on
>>> > > > > > > changes on top of those RPC/record
>>> > > > > > > (AddPartitionsToTxnRequest/TransactionLogValue) changes made
>>> in
>>> > > > KIP-890
>>> > > > > > > will be automatically downgraded too?
>>> > > > > > >
>>> > > > > > > Jun
>>> > > > > > >
>>> > > > > > > On Tue, Jan 30, 2024 at 3:32 PM Justine Olshan
>>> > > > > > > <jo...@confluent.io.invalid>
>>> > > > > > > wrote:
>>> > > > > > >
>>> > > > > > > > Hey Jun,
>>> > > > > > > >
>>> > > > > > > > I wanted to get back to you about your questions about
>>> MV/IBP.
>>> > > > > > > >
>>> > > > > > > > Looking at the options, I think it makes the most sense to
>>> > > create a
>>> > > > > > > > separate feature for transactions and use that to version
>>> gate
>>> > > the
>>> > > > > > > features
>>> > > > > > > > we need to version gate (flexible transactional state
>>> records
>>> > and
>>> > > > > using
>>> > > > > > > the
>>> > > > > > > > new protocol)
>>> > > > > > > > I've updated the KIP to include this change. Hopefully
>>> that's
>>> > > > > > everything
>>> > > > > > > we
>>> > > > > > > > need for this KIP :)
>>> > > > > > > >
>>> > > > > > > > Justine
>>> > > > > > > >
>>> > > > > > > >
>>> > > > > > > > On Mon, Jan 22, 2024 at 3:17 PM Justine Olshan <
>>> > > > jolshan@confluent.io
>>> > > > > >
>>> > > > > > > > wrote:
>>> > > > > > > >
>>> > > > > > > > > Thanks Jun,
>>> > > > > > > > >
>>> > > > > > > > > I will update the KIP with the prev field for prepare as
>>> > well.
>>> > > > > > > > >
>>> > > > > > > > > PREPARE
>>> > > > > > > > > producerId: x
>>> > > > > > > > > previous/lastProducerId (tagged field): x
>>> > > > > > > > > nextProducerId (tagged field): empty or z if y will
>>> overflow
>>> > > > > > > > > producerEpoch: y + 1
>>> > > > > > > > >
>>> > > > > > > > > COMPLETE
>>> > > > > > > > > producerId: x or z if y overflowed
>>> > > > > > > > > previous/lastProducerId (tagged field): x
>>> > > > > > > > > nextProducerId (tagged field): empty
>>> > > > > > > > > producerEpoch: y + 1 or 0 if we overflowed
>>> > > > > > > > >
>>> > > > > > > > > Thanks again,
>>> > > > > > > > > Justine
>>> > > > > > > > >
>>> > > > > > > > > On Mon, Jan 22, 2024 at 3:15 PM Jun Rao
>>> > > <jun@confluent.io.invalid
>>> > > > >
>>> > > > > > > > wrote:
>>> > > > > > > > >
>>> > > > > > > > >> Hi, Justine,
>>> > > > > > > > >>
>>> > > > > > > > >> 101.3 Thanks for the explanation.
>>> > > > > > > > >> (1) My point was that the coordinator could fail right
>>> after
>>> > > > > writing
>>> > > > > > > the
>>> > > > > > > > >> prepare marker. When the new txn coordinator generates
>>> the
>>> > > > > complete
>>> > > > > > > > marker
>>> > > > > > > > >> after the failover, it needs some field from the prepare
>>> > > marker
>>> > > > to
>>> > > > > > > > >> determine whether it's written by the new client.
>>> > > > > > > > >>
>>> > > > > > > > >> (2) The changing of the behavior sounds good to me. We
>>> only
>>> > > want
>>> > > > > to
>>> > > > > > > > return
>>> > > > > > > > >> success if the prepare state is written by the new
>>> client.
>>> > So,
>>> > > > in
>>> > > > > > the
>>> > > > > > > > >> non-overflow case, it seems that we also need sth in the
>>> > > prepare
>>> > > > > > > marker
>>> > > > > > > > to
>>> > > > > > > > >> tell us whether it's written by the new client.
>>> > > > > > > > >>
>>> > > > > > > > >> 112. Thanks for the explanation. That sounds good to me.
>>> > > > > > > > >>
>>> > > > > > > > >> Jun
>>> > > > > > > > >>
>>> > > > > > > > >> On Mon, Jan 22, 2024 at 11:32 AM Justine Olshan
>>> > > > > > > > >> <jo...@confluent.io.invalid> wrote:
>>> > > > > > > > >>
>>> > > > > > > > >> > 101.3 I realized that I actually have two questions.
>>> > > > > > > > >> > > (1) In the non-overflow case, we need to write the
>>> > > previous
>>> > > > > > > produce
>>> > > > > > > > Id
>>> > > > > > > > >> > tagged field in the end maker so that we know if the
>>> > marker
>>> > > is
>>> > > > > > from
>>> > > > > > > > the
>>> > > > > > > > >> new
>>> > > > > > > > >> > client. Since the end maker is derived from the
>>> prepare
>>> > > > marker,
>>> > > > > > > should
>>> > > > > > > > >> we
>>> > > > > > > > >> > write the previous produce Id in the prepare marker
>>> field
>>> > > too?
>>> > > > > > > > >> Otherwise,
>>> > > > > > > > >> > we will lose this information when deriving the end
>>> > marker.
>>> > > > > > > > >> >
>>> > > > > > > > >> > The "previous" producer ID is in the normal producer
>>> ID
>>> > > field.
>>> > > > > So
>>> > > > > > > yes,
>>> > > > > > > > >> we
>>> > > > > > > > >> > need it in prepare and that was always the plan.
>>> > > > > > > > >> >
>>> > > > > > > > >> > Maybe it is a bit unclear so I will enumerate the
>>> fields
>>> > and
>>> > > > add
>>> > > > > > > them
>>> > > > > > > > to
>>> > > > > > > > >> > the KIP if that helps.
>>> > > > > > > > >> > Say we have producer ID x and epoch y. When we
>>> overflow
>>> > > epoch
>>> > > > y
>>> > > > > we
>>> > > > > > > get
>>> > > > > > > > >> > producer ID Z.
>>> > > > > > > > >> >
>>> > > > > > > > >> > PREPARE
>>> > > > > > > > >> > producerId: x
>>> > > > > > > > >> > previous/lastProducerId (tagged field): empty
>>> > > > > > > > >> > nextProducerId (tagged field): empty or z if y will
>>> > overflow
>>> > > > > > > > >> > producerEpoch: y + 1
>>> > > > > > > > >> >
>>> > > > > > > > >> > COMPLETE
>>> > > > > > > > >> > producerId: x or z if y overflowed
>>> > > > > > > > >> > previous/lastProducerId (tagged field): x
>>> > > > > > > > >> > nextProducerId (tagged field): empty
>>> > > > > > > > >> > producerEpoch: y + 1 or 0 if we overflowed
>>> > > > > > > > >> >
>>> > > > > > > > >> > (2) In the prepare phase, if we retry and see epoch -
>>> 1 +
>>> > ID
>>> > > > in
>>> > > > > > last
>>> > > > > > > > >> seen
>>> > > > > > > > >> > fields and are issuing the same command (ie commit not
>>> > > abort),
>>> > > > > we
>>> > > > > > > > return
>>> > > > > > > > >> > success. The logic before KIP-890 seems to return
>>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>>> > > > > > > > >> > in this case. Are we intentionally making this change?
>>> > > > > > > > >> >
>>> > > > > > > > >> > Hmm -- we would fence the producer if the epoch is
>>> bumped
>>> > > and
>>> > > > we
>>> > > > > > > get a
>>> > > > > > > > >> > lower epoch. Yes -- we are intentionally adding this
>>> to
>>> > > > prevent
>>> > > > > > > > fencing.
>>> > > > > > > > >> >
>>> > > > > > > > >> >
>>> > > > > > > > >> > 112. We already merged the code that adds the
>>> VerifyOnly
>>> > > field
>>> > > > > in
>>> > > > > > > > >> > AddPartitionsToTxnRequest, which is an inter broker
>>> > request.
>>> > > > It
>>> > > > > > > seems
>>> > > > > > > > >> that
>>> > > > > > > > >> > we didn't bump up the IBP for that. Do you know why?
>>> > > > > > > > >> >
>>> > > > > > > > >> > We no longer need IBP for all interbroker requests as
>>> > > > > ApiVersions
>>> > > > > > > > should
>>> > > > > > > > >> > correctly gate versioning.
>>> > > > > > > > >> > We also handle unsupported version errors correctly
>>> if we
>>> > > > > receive
>>> > > > > > > them
>>> > > > > > > > >> in
>>> > > > > > > > >> > edge cases like upgrades/downgrades.
>>> > > > > > > > >> >
>>> > > > > > > > >> > Justine
>>> > > > > > > > >> >
>>> > > > > > > > >> > On Mon, Jan 22, 2024 at 11:00 AM Jun Rao
>>> > > > > <jun@confluent.io.invalid
>>> > > > > > >
>>> > > > > > > > >> wrote:
>>> > > > > > > > >> >
>>> > > > > > > > >> > > Hi, Justine,
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > Thanks for the reply.
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > 101.3 I realized that I actually have two questions.
>>> > > > > > > > >> > > (1) In the non-overflow case, we need to write the
>>> > > previous
>>> > > > > > > produce
>>> > > > > > > > Id
>>> > > > > > > > >> > > tagged field in the end maker so that we know if the
>>> > > marker
>>> > > > is
>>> > > > > > > from
>>> > > > > > > > >> the
>>> > > > > > > > >> > new
>>> > > > > > > > >> > > client. Since the end maker is derived from the
>>> prepare
>>> > > > > marker,
>>> > > > > > > > >> should we
>>> > > > > > > > >> > > write the previous produce Id in the prepare marker
>>> > field
>>> > > > too?
>>> > > > > > > > >> Otherwise,
>>> > > > > > > > >> > > we will lose this information when deriving the end
>>> > > marker.
>>> > > > > > > > >> > > (2) In the prepare phase, if we retry and see epoch
>>> - 1
>>> > +
>>> > > ID
>>> > > > > in
>>> > > > > > > last
>>> > > > > > > > >> seen
>>> > > > > > > > >> > > fields and are issuing the same command (ie commit
>>> not
>>> > > > abort),
>>> > > > > > we
>>> > > > > > > > >> return
>>> > > > > > > > >> > > success. The logic before KIP-890 seems to return
>>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>>> > > > > > > > >> > > in this case. Are we intentionally making this
>>> change?
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > 112. We already merged the code that adds the
>>> VerifyOnly
>>> > > > field
>>> > > > > > in
>>> > > > > > > > >> > > AddPartitionsToTxnRequest, which is an inter broker
>>> > > request.
>>> > > > > It
>>> > > > > > > > seems
>>> > > > > > > > >> > that
>>> > > > > > > > >> > > we didn't bump up the IBP for that. Do you know why?
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > Jun
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > On Fri, Jan 19, 2024 at 4:50 PM Justine Olshan
>>> > > > > > > > >> > > <jo...@confluent.io.invalid>
>>> > > > > > > > >> > > wrote:
>>> > > > > > > > >> > >
>>> > > > > > > > >> > > > Hi Jun,
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > > > 101.3 I can change "last seen" to "current
>>> producer id
>>> > > and
>>> > > > > > > epoch"
>>> > > > > > > > if
>>> > > > > > > > >> > that
>>> > > > > > > > >> > > > was the part that was confusing
>>> > > > > > > > >> > > > 110 I can mention this
>>> > > > > > > > >> > > > 111 I can do that
>>> > > > > > > > >> > > > 112 We still need it. But I am still finalizing
>>> the
>>> > > > design.
>>> > > > > I
>>> > > > > > > will
>>> > > > > > > > >> > update
>>> > > > > > > > >> > > > the KIP once I get the information finalized.
>>> Sorry
>>> > for
>>> > > > the
>>> > > > > > > > delays.
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > > > Justine
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > > > On Fri, Jan 19, 2024 at 10:50 AM Jun Rao
>>> > > > > > > <jun@confluent.io.invalid
>>> > > > > > > > >
>>> > > > > > > > >> > > wrote:
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > > > > Hi, Justine,
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > Thanks for the reply.
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > 101.3 In the non-overflow case, the previous ID
>>> is
>>> > the
>>> > > > > same
>>> > > > > > as
>>> > > > > > > > the
>>> > > > > > > > >> > > > produce
>>> > > > > > > > >> > > > > ID for the complete marker too, but we set the
>>> > > previous
>>> > > > ID
>>> > > > > > in
>>> > > > > > > > the
>>> > > > > > > > >> > > > complete
>>> > > > > > > > >> > > > > marker. Earlier you mentioned that this is to
>>> know
>>> > > that
>>> > > > > the
>>> > > > > > > > >> marker is
>>> > > > > > > > >> > > > > written by the new client so that we could
>>> return
>>> > > > success
>>> > > > > on
>>> > > > > > > > >> retried
>>> > > > > > > > >> > > > > endMarker requests. I was trying to understand
>>> why
>>> > > this
>>> > > > is
>>> > > > > > not
>>> > > > > > > > >> needed
>>> > > > > > > > >> > > for
>>> > > > > > > > >> > > > > the prepare marker since retry can happen in the
>>> > > prepare
>>> > > > > > state
>>> > > > > > > > >> too.
>>> > > > > > > > >> > Is
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > reason that in the prepare state, we return
>>> > > > > > > > >> CONCURRENT_TRANSACTIONS
>>> > > > > > > > >> > > > instead
>>> > > > > > > > >> > > > > of success on retried endMaker requests? If so,
>>> > should
>>> > > > we
>>> > > > > > > change
>>> > > > > > > > >> "If
>>> > > > > > > > >> > we
>>> > > > > > > > >> > > > > retry and see epoch - 1 + ID in last seen
>>> fields and
>>> > > are
>>> > > > > > > issuing
>>> > > > > > > > >> the
>>> > > > > > > > >> > > same
>>> > > > > > > > >> > > > > command (ie commit not abort) we can return
>>> (with
>>> > the
>>> > > > new
>>> > > > > > > > epoch)"
>>> > > > > > > > >> > > > > accordingly?
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > 110. Yes, without this KIP, a delayed endMaker
>>> > request
>>> > > > > > carries
>>> > > > > > > > the
>>> > > > > > > > >> > same
>>> > > > > > > > >> > > > > epoch and won't be fenced. This can
>>> commit/abort a
>>> > > > future
>>> > > > > > > > >> transaction
>>> > > > > > > > >> > > > > unexpectedly. I am not sure if we have seen
>>> this in
>>> > > > > practice
>>> > > > > > > > >> though.
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > 111. Sounds good. It would be useful to make it
>>> > clear
>>> > > > that
>>> > > > > > we
>>> > > > > > > > can
>>> > > > > > > > >> now
>>> > > > > > > > >> > > > > populate the lastSeen field from the log
>>> reliably.
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > 112. Yes, I was referring to
>>> > AddPartitionsToTxnRequest
>>> > > > > since
>>> > > > > > > > it's
>>> > > > > > > > >> > > called
>>> > > > > > > > >> > > > > across brokers and we are changing its schema.
>>> Are
>>> > you
>>> > > > > > saying
>>> > > > > > > we
>>> > > > > > > > >> > don't
>>> > > > > > > > >> > > > need
>>> > > > > > > > >> > > > > it any more? I thought that we already
>>> implemented
>>> > the
>>> > > > > > server
>>> > > > > > > > side
>>> > > > > > > > >> > > > > verification logic based on
>>> > AddPartitionsToTxnRequest
>>> > > > > across
>>> > > > > > > > >> brokers.
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > Jun
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > On Thu, Jan 18, 2024 at 5:05 PM Justine Olshan
>>> > > > > > > > >> > > > > <jo...@confluent.io.invalid>
>>> > > > > > > > >> > > > > wrote:
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > > > > Hey Jun,
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > 101.3 We don't set the previous ID in the
>>> Prepare
>>> > > > field
>>> > > > > > > since
>>> > > > > > > > we
>>> > > > > > > > >> > > don't
>>> > > > > > > > >> > > > > need
>>> > > > > > > > >> > > > > > it. It is the same producer ID as the main
>>> > producer
>>> > > ID
>>> > > > > > > field.
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > 110 Hmm -- maybe I need to reread your message
>>> > about
>>> > > > > > delayed
>>> > > > > > > > >> > markers.
>>> > > > > > > > >> > > > If
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > receive a delayed endTxn marker after the
>>> > > transaction
>>> > > > is
>>> > > > > > > > already
>>> > > > > > > > >> > > > > complete?
>>> > > > > > > > >> > > > > > So we will commit the next transaction early
>>> > without
>>> > > > the
>>> > > > > > > fixes
>>> > > > > > > > >> in
>>> > > > > > > > >> > > part
>>> > > > > > > > >> > > > 2?
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > 111 Yes -- this terminology was used in a
>>> previous
>>> > > KIP
>>> > > > > and
>>> > > > > > > > never
>>> > > > > > > > >> > > > > > implemented it in the log -- only in memory
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > 112 Hmm -- which interbroker protocol are you
>>> > > > referring
>>> > > > > > to?
>>> > > > > > > I
>>> > > > > > > > am
>>> > > > > > > > >> > > > working
>>> > > > > > > > >> > > > > on
>>> > > > > > > > >> > > > > > the design for the work to remove the extra
>>> add
>>> > > > > partitions
>>> > > > > > > > call
>>> > > > > > > > >> > and I
>>> > > > > > > > >> > > > > right
>>> > > > > > > > >> > > > > > now the design bumps MV. I have yet to update
>>> that
>>> > > > > section
>>> > > > > > > as
>>> > > > > > > > I
>>> > > > > > > > >> > > > finalize
>>> > > > > > > > >> > > > > > the design so please stay tuned. Was there
>>> > anything
>>> > > > else
>>> > > > > > you
>>> > > > > > > > >> > thought
>>> > > > > > > > >> > > > > needed
>>> > > > > > > > >> > > > > > MV bump?
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > Justine
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > On Thu, Jan 18, 2024 at 3:07 PM Jun Rao
>>> > > > > > > > >> <ju...@confluent.io.invalid>
>>> > > > > > > > >> > > > > wrote:
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > I don't see this create any issue. It just
>>> makes
>>> > > it
>>> > > > a
>>> > > > > > bit
>>> > > > > > > > >> hard to
>>> > > > > > > > >> > > > > explain
>>> > > > > > > > >> > > > > > > what this non-tagged produce id field
>>> means. We
>>> > > are
>>> > > > > > > > >> essentially
>>> > > > > > > > >> > > > trying
>>> > > > > > > > >> > > > > to
>>> > > > > > > > >> > > > > > > combine two actions (completing a txn and
>>> init a
>>> > > new
>>> > > > > > > produce
>>> > > > > > > > >> Id)
>>> > > > > > > > >> > > in a
>>> > > > > > > > >> > > > > > > single record. But, this may be fine too.
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > A few other follow up comments.
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > 101.3 I guess the reason that we only set
>>> the
>>> > > > previous
>>> > > > > > > > >> produce id
>>> > > > > > > > >> > > > > tagged
>>> > > > > > > > >> > > > > > > field in the complete marker, but not in the
>>> > > prepare
>>> > > > > > > marker,
>>> > > > > > > > >> is
>>> > > > > > > > >> > > that
>>> > > > > > > > >> > > > in
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > prepare state, we always return
>>> > > > > CONCURRENT_TRANSACTIONS
>>> > > > > > on
>>> > > > > > > > >> > retried
>>> > > > > > > > >> > > > > > endMaker
>>> > > > > > > > >> > > > > > > requests?
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > 110. "I believe your second point is
>>> mentioned
>>> > in
>>> > > > the
>>> > > > > > > KIP. I
>>> > > > > > > > >> can
>>> > > > > > > > >> > > add
>>> > > > > > > > >> > > > > more
>>> > > > > > > > >> > > > > > > text on
>>> > > > > > > > >> > > > > > > this if it is helpful.
>>> > > > > > > > >> > > > > > > > The delayed message case can also violate
>>> EOS
>>> > if
>>> > > > the
>>> > > > > > > > delayed
>>> > > > > > > > >> > > > message
>>> > > > > > > > >> > > > > > > comes in after the next addPartitionsToTxn
>>> > request
>>> > > > > comes
>>> > > > > > > in.
>>> > > > > > > > >> > > > > Effectively
>>> > > > > > > > >> > > > > > we
>>> > > > > > > > >> > > > > > > may see a message from a previous (aborted)
>>> > > > > transaction
>>> > > > > > > > become
>>> > > > > > > > >> > part
>>> > > > > > > > >> > > > of
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > next transaction."
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > The above is the case when a delayed
>>> message is
>>> > > > > appended
>>> > > > > > > to
>>> > > > > > > > >> the
>>> > > > > > > > >> > > data
>>> > > > > > > > >> > > > > > > partition. What I mentioned is a slightly
>>> > > different
>>> > > > > case
>>> > > > > > > > when
>>> > > > > > > > >> a
>>> > > > > > > > >> > > > delayed
>>> > > > > > > > >> > > > > > > marker is appended to the transaction log
>>> > > partition.
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > 111. The KIP says "Once we move past the
>>> Prepare
>>> > > and
>>> > > > > > > > Complete
>>> > > > > > > > >> > > states,
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > > don’t need to worry about lastSeen fields
>>> and
>>> > > clear
>>> > > > > > them,
>>> > > > > > > > just
>>> > > > > > > > >> > > handle
>>> > > > > > > > >> > > > > > state
>>> > > > > > > > >> > > > > > > transitions as normal.". Is the lastSeen
>>> field
>>> > the
>>> > > > > same
>>> > > > > > as
>>> > > > > > > > the
>>> > > > > > > > >> > > > previous
>>> > > > > > > > >> > > > > > > Produce Id tagged field in
>>> TransactionLogValue?
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > 112. Since the kip changes the inter-broker
>>> > > > protocol,
>>> > > > > > > should
>>> > > > > > > > >> we
>>> > > > > > > > >> > > bump
>>> > > > > > > > >> > > > up
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > MV/IBP version? Is this feature only for the
>>> > KRaft
>>> > > > > mode?
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > Thanks,
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > Jun
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > On Wed, Jan 17, 2024 at 11:13 AM Justine
>>> Olshan
>>> > > > > > > > >> > > > > > > <jo...@confluent.io.invalid> wrote:
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > > > > Hey Jun,
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > > > I'm glad we are getting to convergence on
>>> the
>>> > > > > design.
>>> > > > > > :)
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > > > While I understand it seems a little
>>> "weird".
>>> > > I'm
>>> > > > > not
>>> > > > > > > sure
>>> > > > > > > > >> what
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > benefit
>>> > > > > > > > >> > > > > > > > of writing an extra record to the log.
>>> > > > > > > > >> > > > > > > > Is the concern a tool to describe
>>> transactions
>>> > > > won't
>>> > > > > > > work
>>> > > > > > > > >> (ie,
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > complete
>>> > > > > > > > >> > > > > > > > state is needed to calculate the time
>>> since
>>> > the
>>> > > > > > > > transaction
>>> > > > > > > > >> > > > > completed?)
>>> > > > > > > > >> > > > > > > > If we have a reason like this, it is
>>> enough to
>>> > > > > > convince
>>> > > > > > > me
>>> > > > > > > > >> we
>>> > > > > > > > >> > > need
>>> > > > > > > > >> > > > > such
>>> > > > > > > > >> > > > > > > an
>>> > > > > > > > >> > > > > > > > extra record. It seems like it would be
>>> > > replacing
>>> > > > > the
>>> > > > > > > > record
>>> > > > > > > > >> > > > written
>>> > > > > > > > >> > > > > on
>>> > > > > > > > >> > > > > > > > InitProducerId. Is this correct?
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > > > Thanks,
>>> > > > > > > > >> > > > > > > > Justine
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > > > On Tue, Jan 16, 2024 at 5:14 PM Jun Rao
>>> > > > > > > > >> > <jun@confluent.io.invalid
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > > > > > > wrote:
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > Thanks for the explanation. I
>>> understand the
>>> > > > > > intention
>>> > > > > > > > >> now.
>>> > > > > > > > >> > In
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > overflow
>>> > > > > > > > >> > > > > > > > > case, we set the non-tagged field to
>>> the old
>>> > > pid
>>> > > > > > (and
>>> > > > > > > > the
>>> > > > > > > > >> max
>>> > > > > > > > >> > > > > epoch)
>>> > > > > > > > >> > > > > > in
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > prepare marker so that we could
>>> correctly
>>> > > write
>>> > > > > the
>>> > > > > > > > >> marker to
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > data
>>> > > > > > > > >> > > > > > > > > partition if the broker downgrades. When
>>> > > writing
>>> > > > > the
>>> > > > > > > > >> complete
>>> > > > > > > > >> > > > > marker,
>>> > > > > > > > >> > > > > > > we
>>> > > > > > > > >> > > > > > > > > know the marker has already been
>>> written to
>>> > > the
>>> > > > > data
>>> > > > > > > > >> > partition.
>>> > > > > > > > >> > > > We
>>> > > > > > > > >> > > > > > set
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > non-tagged field to the new pid to avoid
>>> > > > > > > > >> > > > InvalidPidMappingException
>>> > > > > > > > >> > > > > > in
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > client if the broker downgrades.
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > The above seems to work. It's just a bit
>>> > > > > > inconsistent
>>> > > > > > > > for
>>> > > > > > > > >> a
>>> > > > > > > > >> > > > prepare
>>> > > > > > > > >> > > > > > > > marker
>>> > > > > > > > >> > > > > > > > > and a complete marker to use different
>>> pids
>>> > in
>>> > > > > this
>>> > > > > > > > >> special
>>> > > > > > > > >> > > case.
>>> > > > > > > > >> > > > > If
>>> > > > > > > > >> > > > > > we
>>> > > > > > > > >> > > > > > > > > downgrade with the complete marker, it
>>> seems
>>> > > > that
>>> > > > > we
>>> > > > > > > > will
>>> > > > > > > > >> > never
>>> > > > > > > > >> > > > be
>>> > > > > > > > >> > > > > > able
>>> > > > > > > > >> > > > > > > > to
>>> > > > > > > > >> > > > > > > > > write the complete marker with the old
>>> pid.
>>> > > Not
>>> > > > > sure
>>> > > > > > > if
>>> > > > > > > > it
>>> > > > > > > > >> > > causes
>>> > > > > > > > >> > > > > any
>>> > > > > > > > >> > > > > > > > > issue, but it seems a bit weird.
>>> Instead of
>>> > > > > writing
>>> > > > > > > the
>>> > > > > > > > >> > > complete
>>> > > > > > > > >> > > > > > marker
>>> > > > > > > > >> > > > > > > > > with the new pid, could we write two
>>> > records:
>>> > > a
>>> > > > > > > complete
>>> > > > > > > > >> > marker
>>> > > > > > > > >> > > > > with
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > old pid followed by a
>>> TransactionLogValue
>>> > with
>>> > > > the
>>> > > > > > new
>>> > > > > > > > pid
>>> > > > > > > > >> > and
>>> > > > > > > > >> > > an
>>> > > > > > > > >> > > > > > empty
>>> > > > > > > > >> > > > > > > > > state? We could make the two records in
>>> the
>>> > > same
>>> > > > > > batch
>>> > > > > > > > so
>>> > > > > > > > >> > that
>>> > > > > > > > >> > > > they
>>> > > > > > > > >> > > > > > > will
>>> > > > > > > > >> > > > > > > > be
>>> > > > > > > > >> > > > > > > > > added to the log atomically.
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > Thanks,
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > Jun
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > On Fri, Jan 12, 2024 at 5:40 PM Justine
>>> > Olshan
>>> > > > > > > > >> > > > > > > > > <jo...@confluent.io.invalid>
>>> > > > > > > > >> > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > (1) the prepare marker is written,
>>> but the
>>> > > > > endTxn
>>> > > > > > > > >> response
>>> > > > > > > > >> > is
>>> > > > > > > > >> > > > not
>>> > > > > > > > >> > > > > > > > > received
>>> > > > > > > > >> > > > > > > > > > by the client when the server
>>> downgrades
>>> > > > > > > > >> > > > > > > > > > (2)  the prepare marker is written,
>>> the
>>> > > endTxn
>>> > > > > > > > response
>>> > > > > > > > >> is
>>> > > > > > > > >> > > > > received
>>> > > > > > > > >> > > > > > > by
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > client when the server downgrades.
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > I think I am still a little confused.
>>> In
>>> > > both
>>> > > > of
>>> > > > > > > these
>>> > > > > > > > >> > cases,
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > > > transaction log has the old producer
>>> ID.
>>> > We
>>> > > > > don't
>>> > > > > > > > write
>>> > > > > > > > >> the
>>> > > > > > > > >> > > new
>>> > > > > > > > >> > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > in the prepare marker's non tagged
>>> fields.
>>> > > > > > > > >> > > > > > > > > > If the server downgrades now, it would
>>> > read
>>> > > > the
>>> > > > > > > > records
>>> > > > > > > > >> not
>>> > > > > > > > >> > > in
>>> > > > > > > > >> > > > > > tagged
>>> > > > > > > > >> > > > > > > > > > fields and the complete marker will
>>> also
>>> > > have
>>> > > > > the
>>> > > > > > > old
>>> > > > > > > > >> > > producer
>>> > > > > > > > >> > > > > ID.
>>> > > > > > > > >> > > > > > > > > > (If we had used the new producer ID,
>>> we
>>> > > would
>>> > > > > not
>>> > > > > > > have
>>> > > > > > > > >> > > > > > transactional
>>> > > > > > > > >> > > > > > > > > > correctness since the producer id
>>> doesn't
>>> > > > match
>>> > > > > > the
>>> > > > > > > > >> > > transaction
>>> > > > > > > > >> > > > > and
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > > state would not be correct on the data
>>> > > > > partition.)
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > In the overflow case, I'd expect the
>>> > > following
>>> > > > > to
>>> > > > > > > > >> happen on
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > client
>>> > > > > > > > >> > > > > > > > > side
>>> > > > > > > > >> > > > > > > > > > Case 1  -- we retry EndTxn -- it is
>>> the
>>> > same
>>> > > > > > > producer
>>> > > > > > > > ID
>>> > > > > > > > >> > and
>>> > > > > > > > >> > > > > epoch
>>> > > > > > > > >> > > > > > -
>>> > > > > > > > >> > > > > > > 1
>>> > > > > > > > >> > > > > > > > > this
>>> > > > > > > > >> > > > > > > > > > would fence the producer
>>> > > > > > > > >> > > > > > > > > > Case 2 -- we don't retry EndTxn and
>>> use
>>> > the
>>> > > > new
>>> > > > > > > > >> producer id
>>> > > > > > > > >> > > > which
>>> > > > > > > > >> > > > > > > would
>>> > > > > > > > >> > > > > > > > > > result in InvalidPidMappingException
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > Maybe we can have special handling for
>>> > when
>>> > > a
>>> > > > > > server
>>> > > > > > > > >> > > > downgrades.
>>> > > > > > > > >> > > > > > When
>>> > > > > > > > >> > > > > > > > it
>>> > > > > > > > >> > > > > > > > > > reconnects we could get an API version
>>> > > request
>>> > > > > > > showing
>>> > > > > > > > >> > > KIP-890
>>> > > > > > > > >> > > > > > part 2
>>> > > > > > > > >> > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > not supported. In that case, we can
>>> call
>>> > > > > > > > initProducerId
>>> > > > > > > > >> to
>>> > > > > > > > >> > > > abort
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > > > transaction. (In the overflow case,
>>> this
>>> > > > > correctly
>>> > > > > > > > gives
>>> > > > > > > > >> > us a
>>> > > > > > > > >> > > > new
>>> > > > > > > > >> > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > ID)
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > I guess the corresponding case would
>>> be
>>> > > where
>>> > > > > the
>>> > > > > > > > >> *complete
>>> > > > > > > > >> > > > > marker
>>> > > > > > > > >> > > > > > > *is
>>> > > > > > > > >> > > > > > > > > > written but the endTxn is not
>>> received by
>>> > > the
>>> > > > > > client
>>> > > > > > > > and
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > > server
>>> > > > > > > > >> > > > > > > > > > downgrades? This would result in the
>>> > > > transaction
>>> > > > > > > > >> > coordinator
>>> > > > > > > > >> > > > > having
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > new
>>> > > > > > > > >> > > > > > > > > > ID and not the old one.  If the client
>>> > > > retries,
>>> > > > > it
>>> > > > > > > > will
>>> > > > > > > > >> > > receive
>>> > > > > > > > >> > > > > an
>>> > > > > > > > >> > > > > > > > > > InvalidPidMappingException. The
>>> > > InitProducerId
>>> > > > > > > > scenario
>>> > > > > > > > >> > above
>>> > > > > > > > >> > > > > would
>>> > > > > > > > >> > > > > > > > help
>>> > > > > > > > >> > > > > > > > > > here too.
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > To be clear, my compatibility story is
>>> > meant
>>> > > > to
>>> > > > > > > > support
>>> > > > > > > > >> > > > > downgrades
>>> > > > > > > > >> > > > > > > > server
>>> > > > > > > > >> > > > > > > > > > side in keeping the transactional
>>> > > correctness.
>>> > > > > > > Keeping
>>> > > > > > > > >> the
>>> > > > > > > > >> > > > client
>>> > > > > > > > >> > > > > > > from
>>> > > > > > > > >> > > > > > > > > > fencing itself is not the priority.
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > Hope this helps. I can also add text
>>> in
>>> > the
>>> > > > KIP
>>> > > > > > > about
>>> > > > > > > > >> > > > > > InitProducerId
>>> > > > > > > > >> > > > > > > if
>>> > > > > > > > >> > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > think that fixes some edge cases.
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > Justine
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > On Fri, Jan 12, 2024 at 4:10 PM Jun
>>> Rao
>>> > > > > > > > >> > > > <jun@confluent.io.invalid
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > Thanks for the reply.
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > I agree that we don't need to
>>> optimize
>>> > for
>>> > > > > > fencing
>>> > > > > > > > >> during
>>> > > > > > > > >> > > > > > > downgrades.
>>> > > > > > > > >> > > > > > > > > > > Regarding consistency, there are two
>>> > > > possible
>>> > > > > > > cases:
>>> > > > > > > > >> (1)
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > prepare
>>> > > > > > > > >> > > > > > > > > > marker
>>> > > > > > > > >> > > > > > > > > > > is written, but the endTxn response
>>> is
>>> > not
>>> > > > > > > received
>>> > > > > > > > by
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > > client
>>> > > > > > > > >> > > > > > > > when
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > server downgrades; (2)  the prepare
>>> > marker
>>> > > > is
>>> > > > > > > > written,
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > > endTxn
>>> > > > > > > > >> > > > > > > > > > response
>>> > > > > > > > >> > > > > > > > > > > is received by the client when the
>>> > server
>>> > > > > > > > downgrades.
>>> > > > > > > > >> In
>>> > > > > > > > >> > > (1),
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > > client
>>> > > > > > > > >> > > > > > > > > > > will have the old produce Id and in
>>> (2),
>>> > > the
>>> > > > > > > client
>>> > > > > > > > >> will
>>> > > > > > > > >> > > have
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > new
>>> > > > > > > > >> > > > > > > > > > > produce Id. If we downgrade right
>>> after
>>> > > the
>>> > > > > > > prepare
>>> > > > > > > > >> > marker,
>>> > > > > > > > >> > > > we
>>> > > > > > > > >> > > > > > > can't
>>> > > > > > > > >> > > > > > > > be
>>> > > > > > > > >> > > > > > > > > > > consistent to both (1) and (2)
>>> since we
>>> > > can
>>> > > > > only
>>> > > > > > > put
>>> > > > > > > > >> one
>>> > > > > > > > >> > > > value
>>> > > > > > > > >> > > > > in
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > existing produce Id field. It's
>>> also not
>>> > > > clear
>>> > > > > > > which
>>> > > > > > > > >> case
>>> > > > > > > > >> > > is
>>> > > > > > > > >> > > > > more
>>> > > > > > > > >> > > > > > > > > likely.
>>> > > > > > > > >> > > > > > > > > > > So we could probably be consistent
>>> with
>>> > > > either
>>> > > > > > > case.
>>> > > > > > > > >> By
>>> > > > > > > > >> > > > putting
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > new
>>> > > > > > > > >> > > > > > > > > > > producer Id in the prepare marker,
>>> we
>>> > are
>>> > > > > > > consistent
>>> > > > > > > > >> with
>>> > > > > > > > >> > > > case
>>> > > > > > > > >> > > > > > (2)
>>> > > > > > > > >> > > > > > > > and
>>> > > > > > > > >> > > > > > > > > it
>>> > > > > > > > >> > > > > > > > > > > also has the slight benefit that the
>>> > > produce
>>> > > > > > field
>>> > > > > > > > in
>>> > > > > > > > >> the
>>> > > > > > > > >> > > > > prepare
>>> > > > > > > > >> > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > complete marker are consistent in
>>> the
>>> > > > overflow
>>> > > > > > > case.
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > Jun
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > On Fri, Jan 12, 2024 at 3:11 PM
>>> Justine
>>> > > > Olshan
>>> > > > > > > > >> > > > > > > > > > > <jo...@confluent.io.invalid>
>>> > > > > > > > >> > > > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > Hi Jun,
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > In the case you describe, we would
>>> > need
>>> > > to
>>> > > > > > have
>>> > > > > > > a
>>> > > > > > > > >> > delayed
>>> > > > > > > > >> > > > > > > request,
>>> > > > > > > > >> > > > > > > > > > send a
>>> > > > > > > > >> > > > > > > > > > > > successful EndTxn, and a
>>> successful
>>> > > > > > > > >> AddPartitionsToTxn
>>> > > > > > > > >> > > and
>>> > > > > > > > >> > > > > then
>>> > > > > > > > >> > > > > > > > have
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > delayed EndTxn request go through
>>> for
>>> > a
>>> > > > > given
>>> > > > > > > > >> producer.
>>> > > > > > > > >> > > > > > > > > > > > I'm trying to figure out if it is
>>> > > possible
>>> > > > > for
>>> > > > > > > the
>>> > > > > > > > >> > client
>>> > > > > > > > >> > > > to
>>> > > > > > > > >> > > > > > > > > transition
>>> > > > > > > > >> > > > > > > > > > > if
>>> > > > > > > > >> > > > > > > > > > > > a previous request is delayed
>>> > somewhere.
>>> > > > But
>>> > > > > > > yes,
>>> > > > > > > > in
>>> > > > > > > > >> > this
>>> > > > > > > > >> > > > > case
>>> > > > > > > > >> > > > > > I
>>> > > > > > > > >> > > > > > > > > think
>>> > > > > > > > >> > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > would fence the client.
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > Not for the overflow case. In the
>>> > > overflow
>>> > > > > > case,
>>> > > > > > > > the
>>> > > > > > > > >> > > > producer
>>> > > > > > > > >> > > > > > ID
>>> > > > > > > > >> > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > epoch are different on the marker
>>> and
>>> > on
>>> > > > the
>>> > > > > > new
>>> > > > > > > > >> > > > transaction.
>>> > > > > > > > >> > > > > > So
>>> > > > > > > > >> > > > > > > we
>>> > > > > > > > >> > > > > > > > > > want
>>> > > > > > > > >> > > > > > > > > > > > the marker to use the max epoch
>>> but
>>> > the
>>> > > > new
>>> > > > > > > > >> > transaction
>>> > > > > > > > >> > > > > should
>>> > > > > > > > >> > > > > > > > start
>>> > > > > > > > >> > > > > > > > > > > with
>>> > > > > > > > >> > > > > > > > > > > > the new ID and epoch 0 in the
>>> > > > transactional
>>> > > > > > > state.
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > In the server downgrade case, we
>>> want
>>> > to
>>> > > > see
>>> > > > > > the
>>> > > > > > > > >> > producer
>>> > > > > > > > >> > > > ID
>>> > > > > > > > >> > > > > as
>>> > > > > > > > >> > > > > > > > that
>>> > > > > > > > >> > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > what the client will have. If we
>>> > > complete
>>> > > > > the
>>> > > > > > > > >> commit,
>>> > > > > > > > >> > and
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > > > transaction
>>> > > > > > > > >> > > > > > > > > > > > state is reloaded, we need the new
>>> > > > producer
>>> > > > > ID
>>> > > > > > > in
>>> > > > > > > > >> the
>>> > > > > > > > >> > > state
>>> > > > > > > > >> > > > > so
>>> > > > > > > > >> > > > > > > > there
>>> > > > > > > > >> > > > > > > > > > > isn't
>>> > > > > > > > >> > > > > > > > > > > > an invalid producer ID mapping.
>>> > > > > > > > >> > > > > > > > > > > > The server downgrade cases are
>>> > > considering
>>> > > > > > > > >> > transactional
>>> > > > > > > > >> > > > > > > > correctness
>>> > > > > > > > >> > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > not regressing from previous
>>> behavior
>>> > --
>>> > > > and
>>> > > > > > are
>>> > > > > > > > not
>>> > > > > > > > >> > > > > concerned
>>> > > > > > > > >> > > > > > > > about
>>> > > > > > > > >> > > > > > > > > > > > supporting the safety from fencing
>>> > > retries
>>> > > > > (as
>>> > > > > > > we
>>> > > > > > > > >> have
>>> > > > > > > > >> > > > > > downgraded
>>> > > > > > > > >> > > > > > > > so
>>> > > > > > > > >> > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > don't need to support). Perhaps
>>> this
>>> > is
>>> > > a
>>> > > > > > trade
>>> > > > > > > > off,
>>> > > > > > > > >> > but
>>> > > > > > > > >> > > I
>>> > > > > > > > >> > > > > > think
>>> > > > > > > > >> > > > > > > it
>>> > > > > > > > >> > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > right one.
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > (If the client downgrades, it will
>>> > have
>>> > > > > > > restarted
>>> > > > > > > > >> and
>>> > > > > > > > >> > it
>>> > > > > > > > >> > > is
>>> > > > > > > > >> > > > > ok
>>> > > > > > > > >> > > > > > > for
>>> > > > > > > > >> > > > > > > > it
>>> > > > > > > > >> > > > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > have a new producer ID too).
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > Justine
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > On Fri, Jan 12, 2024 at 11:42 AM
>>> Jun
>>> > Rao
>>> > > > > > > > >> > > > > > > <jun@confluent.io.invalid
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > Thanks for the reply.
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > 101.4 "If the marker is written
>>> by
>>> > the
>>> > > > new
>>> > > > > > > > >> client, we
>>> > > > > > > > >> > > can
>>> > > > > > > > >> > > > > as
>>> > > > > > > > >> > > > > > I
>>> > > > > > > > >> > > > > > > > > > > mentioned
>>> > > > > > > > >> > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > the last email guarantee that
>>> any
>>> > > EndTxn
>>> > > > > > > > requests
>>> > > > > > > > >> > with
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > same
>>> > > > > > > > >> > > > > > > > > epoch
>>> > > > > > > > >> > > > > > > > > > > are
>>> > > > > > > > >> > > > > > > > > > > > > from the same producer and the
>>> same
>>> > > > > > > transaction.
>>> > > > > > > > >> Then
>>> > > > > > > > >> > > we
>>> > > > > > > > >> > > > > > don't
>>> > > > > > > > >> > > > > > > > have
>>> > > > > > > > >> > > > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > > return a fenced error but can
>>> handle
>>> > > > > > > gracefully
>>> > > > > > > > as
>>> > > > > > > > >> > > > > described
>>> > > > > > > > >> > > > > > in
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > KIP."
>>> > > > > > > > >> > > > > > > > > > > > > When a delayed EndTnx request is
>>> > > > > processed,
>>> > > > > > > the
>>> > > > > > > > >> txn
>>> > > > > > > > >> > > state
>>> > > > > > > > >> > > > > > could
>>> > > > > > > > >> > > > > > > > be
>>> > > > > > > > >> > > > > > > > > > > > ongoing
>>> > > > > > > > >> > > > > > > > > > > > > for the next txn. I guess in
>>> this
>>> > case
>>> > > > we
>>> > > > > > > still
>>> > > > > > > > >> > return
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > fenced
>>> > > > > > > > >> > > > > > > > > > error
>>> > > > > > > > >> > > > > > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > > > > the delayed request?
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > 102. Sorry, my question was
>>> > > inaccurate.
>>> > > > > What
>>> > > > > > > you
>>> > > > > > > > >> > > > described
>>> > > > > > > > >> > > > > is
>>> > > > > > > > >> > > > > > > > > > accurate.
>>> > > > > > > > >> > > > > > > > > > > > > "The downgrade compatibility I
>>> > mention
>>> > > > is
>>> > > > > > that
>>> > > > > > > > we
>>> > > > > > > > >> > keep
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > same
>>> > > > > > > > >> > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > > and epoch in the main
>>> (non-tagged)
>>> > > > fields
>>> > > > > as
>>> > > > > > > we
>>> > > > > > > > >> did
>>> > > > > > > > >> > > > before
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > code
>>> > > > > > > > >> > > > > > > > > > on
>>> > > > > > > > >> > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > server side." If we want to do
>>> this,
>>> > > it
>>> > > > > > seems
>>> > > > > > > > >> that we
>>> > > > > > > > >> > > > > should
>>> > > > > > > > >> > > > > > > use
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > current produce Id and max
>>> epoch in
>>> > > the
>>> > > > > > > existing
>>> > > > > > > > >> > > > producerId
>>> > > > > > > > >> > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > producerEpoch fields for both
>>> the
>>> > > > prepare
>>> > > > > > and
>>> > > > > > > > the
>>> > > > > > > > >> > > > complete
>>> > > > > > > > >> > > > > > > > marker,
>>> > > > > > > > >> > > > > > > > > > > right?
>>> > > > > > > > >> > > > > > > > > > > > > The downgrade can happen after
>>> the
>>> > > > > complete
>>> > > > > > > > >> marker is
>>> > > > > > > > >> > > > > > written.
>>> > > > > > > > >> > > > > > > > With
>>> > > > > > > > >> > > > > > > > > > > what
>>> > > > > > > > >> > > > > > > > > > > > > you described, the downgraded
>>> > > > coordinator
>>> > > > > > will
>>> > > > > > > > see
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > new
>>> > > > > > > > >> > > > > > > > produce
>>> > > > > > > > >> > > > > > > > > Id
>>> > > > > > > > >> > > > > > > > > > > > > instead of the old one.
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > Jun
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > On Fri, Jan 12, 2024 at 10:44 AM
>>> > > Justine
>>> > > > > > > Olshan
>>> > > > > > > > >> > > > > > > > > > > > > <jo...@confluent.io.invalid>
>>> > wrote:
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > Hi Jun,
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > I can update the description.
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > I believe your second point is
>>> > > > mentioned
>>> > > > > > in
>>> > > > > > > > the
>>> > > > > > > > >> > KIP.
>>> > > > > > > > >> > > I
>>> > > > > > > > >> > > > > can
>>> > > > > > > > >> > > > > > > add
>>> > > > > > > > >> > > > > > > > > more
>>> > > > > > > > >> > > > > > > > > > > > text
>>> > > > > > > > >> > > > > > > > > > > > > on
>>> > > > > > > > >> > > > > > > > > > > > > > this if it is helpful.
>>> > > > > > > > >> > > > > > > > > > > > > > > The delayed message case can
>>> > also
>>> > > > > > violate
>>> > > > > > > > EOS
>>> > > > > > > > >> if
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > delayed
>>> > > > > > > > >> > > > > > > > > > > message
>>> > > > > > > > >> > > > > > > > > > > > > > comes in after the next
>>> > > > > addPartitionsToTxn
>>> > > > > > > > >> request
>>> > > > > > > > >> > > > comes
>>> > > > > > > > >> > > > > > in.
>>> > > > > > > > >> > > > > > > > > > > > Effectively
>>> > > > > > > > >> > > > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > may see a message from a
>>> previous
>>> > > > > > (aborted)
>>> > > > > > > > >> > > transaction
>>> > > > > > > > >> > > > > > > become
>>> > > > > > > > >> > > > > > > > > part
>>> > > > > > > > >> > > > > > > > > > > of
>>> > > > > > > > >> > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > next transaction.
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > If the marker is written by
>>> the
>>> > new
>>> > > > > > client,
>>> > > > > > > we
>>> > > > > > > > >> can
>>> > > > > > > > >> > > as I
>>> > > > > > > > >> > > > > > > > mentioned
>>> > > > > > > > >> > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > last email guarantee that any
>>> > EndTxn
>>> > > > > > > requests
>>> > > > > > > > >> with
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > same
>>> > > > > > > > >> > > > > > > > epoch
>>> > > > > > > > >> > > > > > > > > > are
>>> > > > > > > > >> > > > > > > > > > > > > from
>>> > > > > > > > >> > > > > > > > > > > > > > the same producer and the same
>>> > > > > > transaction.
>>> > > > > > > > >> Then we
>>> > > > > > > > >> > > > don't
>>> > > > > > > > >> > > > > > > have
>>> > > > > > > > >> > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > return
>>> > > > > > > > >> > > > > > > > > > > > > a
>>> > > > > > > > >> > > > > > > > > > > > > > fenced error but can handle
>>> > > gracefully
>>> > > > > as
>>> > > > > > > > >> described
>>> > > > > > > > >> > > in
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > KIP.
>>> > > > > > > > >> > > > > > > > > > > > > > I don't think a boolean is
>>> useful
>>> > > > since
>>> > > > > it
>>> > > > > > > is
>>> > > > > > > > >> > > directly
>>> > > > > > > > >> > > > > > > encoded
>>> > > > > > > > >> > > > > > > > by
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > existence or lack of the
>>> tagged
>>> > > field
>>> > > > > > being
>>> > > > > > > > >> > written.
>>> > > > > > > > >> > > > > > > > > > > > > > In the prepare marker we will
>>> have
>>> > > the
>>> > > > > > same
>>> > > > > > > > >> > producer
>>> > > > > > > > >> > > ID
>>> > > > > > > > >> > > > > in
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > non-tagged
>>> > > > > > > > >> > > > > > > > > > > > > > field. In the Complete state
>>> we
>>> > may
>>> > > > not.
>>> > > > > > > > >> > > > > > > > > > > > > > I'm not sure why the ongoing
>>> state
>>> > > > > matters
>>> > > > > > > for
>>> > > > > > > > >> this
>>> > > > > > > > >> > > > KIP.
>>> > > > > > > > >> > > > > It
>>> > > > > > > > >> > > > > > > > does
>>> > > > > > > > >> > > > > > > > > > > matter
>>> > > > > > > > >> > > > > > > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > > > > > KIP-939.
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > I'm not sure what you are
>>> > referring
>>> > > to
>>> > > > > > about
>>> > > > > > > > >> > writing
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > previous
>>> > > > > > > > >> > > > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > ID in the prepare marker.
>>> This is
>>> > > not
>>> > > > in
>>> > > > > > the
>>> > > > > > > > >> KIP.
>>> > > > > > > > >> > > > > > > > > > > > > > In the overflow case, we
>>> write the
>>> > > > > > > > >> nextProducerId
>>> > > > > > > > >> > in
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > prepare
>>> > > > > > > > >> > > > > > > > > > > state.
>>> > > > > > > > >> > > > > > > > > > > > > > This is so we know what we
>>> > assigned
>>> > > > when
>>> > > > > > we
>>> > > > > > > > >> reload
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > > > > transaction
>>> > > > > > > > >> > > > > > > > > > > log.
>>> > > > > > > > >> > > > > > > > > > > > > > Once we complete, we
>>> transition
>>> > this
>>> > > > ID
>>> > > > > to
>>> > > > > > > the
>>> > > > > > > > >> main
>>> > > > > > > > >> > > > > > > (non-tagged
>>> > > > > > > > >> > > > > > > > > > > field)
>>> > > > > > > > >> > > > > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > > have the previous producer ID
>>> > field
>>> > > > > filled
>>> > > > > > > in.
>>> > > > > > > > >> This
>>> > > > > > > > >> > > is
>>> > > > > > > > >> > > > so
>>> > > > > > > > >> > > > > > we
>>> > > > > > > > >> > > > > > > > can
>>> > > > > > > > >> > > > > > > > > > > > identify
>>> > > > > > > > >> > > > > > > > > > > > > > in a retry case the operation
>>> > > > completed
>>> > > > > > > > >> > successfully
>>> > > > > > > > >> > > > and
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > > > don't
>>> > > > > > > > >> > > > > > > > > > > fence
>>> > > > > > > > >> > > > > > > > > > > > > our
>>> > > > > > > > >> > > > > > > > > > > > > > producer. The downgrade
>>> > > compatibility
>>> > > > I
>>> > > > > > > > mention
>>> > > > > > > > >> is
>>> > > > > > > > >> > > that
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > > keep
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > same
>>> > > > > > > > >> > > > > > > > > > > > > > producer ID and epoch in the
>>> main
>>> > > > > > > (non-tagged)
>>> > > > > > > > >> > fields
>>> > > > > > > > >> > > > as
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > > did
>>> > > > > > > > >> > > > > > > > > > > before
>>> > > > > > > > >> > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > code on the server side. If
>>> the
>>> > > server
>>> > > > > > > > >> downgrades,
>>> > > > > > > > >> > we
>>> > > > > > > > >> > > > are
>>> > > > > > > > >> > > > > > > still
>>> > > > > > > > >> > > > > > > > > > > > > compatible.
>>> > > > > > > > >> > > > > > > > > > > > > > This addresses both the
>>> prepare
>>> > and
>>> > > > > > complete
>>> > > > > > > > >> state
>>> > > > > > > > >> > > > > > > downgrades.
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > Justine
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > On Fri, Jan 12, 2024 at
>>> 10:21 AM
>>> > Jun
>>> > > > Rao
>>> > > > > > > > >> > > > > > > > > <jun@confluent.io.invalid
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > Thanks for the reply. Sorry
>>> for
>>> > > the
>>> > > > > > > delay. I
>>> > > > > > > > >> > have a
>>> > > > > > > > >> > > > few
>>> > > > > > > > >> > > > > > > more
>>> > > > > > > > >> > > > > > > > > > > > comments.
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > 110. I think the motivation
>>> > > section
>>> > > > > > could
>>> > > > > > > be
>>> > > > > > > > >> > > > improved.
>>> > > > > > > > >> > > > > > One
>>> > > > > > > > >> > > > > > > of
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > motivations listed by the
>>> KIP is
>>> > > > "This
>>> > > > > > can
>>> > > > > > > > >> happen
>>> > > > > > > > >> > > > when
>>> > > > > > > > >> > > > > a
>>> > > > > > > > >> > > > > > > > > message
>>> > > > > > > > >> > > > > > > > > > > gets
>>> > > > > > > > >> > > > > > > > > > > > > > stuck
>>> > > > > > > > >> > > > > > > > > > > > > > > or delayed due to networking
>>> > > issues
>>> > > > > or a
>>> > > > > > > > >> network
>>> > > > > > > > >> > > > > > partition,
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > transaction
>>> > > > > > > > >> > > > > > > > > > > > > > > aborts, and then the delayed
>>> > > message
>>> > > > > > > finally
>>> > > > > > > > >> > comes
>>> > > > > > > > >> > > > > in.".
>>> > > > > > > > >> > > > > > > This
>>> > > > > > > > >> > > > > > > > > > seems
>>> > > > > > > > >> > > > > > > > > > > > not
>>> > > > > > > > >> > > > > > > > > > > > > > > very accurate. Without
>>> KIP-890,
>>> > > > > > currently,
>>> > > > > > > > if
>>> > > > > > > > >> the
>>> > > > > > > > >> > > > > > > coordinator
>>> > > > > > > > >> > > > > > > > > > times
>>> > > > > > > > >> > > > > > > > > > > > out
>>> > > > > > > > >> > > > > > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > > > aborts an ongoing
>>> transaction,
>>> > it
>>> > > > > > already
>>> > > > > > > > >> bumps
>>> > > > > > > > >> > up
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > epoch
>>> > > > > > > > >> > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > marker,
>>> > > > > > > > >> > > > > > > > > > > > > > > which prevents the delayed
>>> > produce
>>> > > > > > message
>>> > > > > > > > >> from
>>> > > > > > > > >> > > being
>>> > > > > > > > >> > > > > > added
>>> > > > > > > > >> > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > user
>>> > > > > > > > >> > > > > > > > > > > > > > > partition. What can cause a
>>> > > hanging
>>> > > > > > > > >> transaction
>>> > > > > > > > >> > is
>>> > > > > > > > >> > > > that
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > > completes (either aborts or
>>> > > > commits) a
>>> > > > > > > > >> > transaction
>>> > > > > > > > >> > > > > before
>>> > > > > > > > >> > > > > > > > > > > receiving a
>>> > > > > > > > >> > > > > > > > > > > > > > > successful ack on messages
>>> > > published
>>> > > > > in
>>> > > > > > > the
>>> > > > > > > > >> same
>>> > > > > > > > >> > > txn.
>>> > > > > > > > >> > > > > In
>>> > > > > > > > >> > > > > > > this
>>> > > > > > > > >> > > > > > > > > > case,
>>> > > > > > > > >> > > > > > > > > > > > > it's
>>> > > > > > > > >> > > > > > > > > > > > > > > possible for the delayed
>>> message
>>> > > to
>>> > > > be
>>> > > > > > > > >> appended
>>> > > > > > > > >> > to
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > > partition
>>> > > > > > > > >> > > > > > > > > > > > after
>>> > > > > > > > >> > > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > marker, causing a
>>> transaction to
>>> > > > hang.
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > A similar issue (not
>>> mentioned
>>> > in
>>> > > > the
>>> > > > > > > > >> motivation)
>>> > > > > > > > >> > > > could
>>> > > > > > > > >> > > > > > > > happen
>>> > > > > > > > >> > > > > > > > > on
>>> > > > > > > > >> > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > marker in the coordinator's
>>> log.
>>> > > For
>>> > > > > > > > example,
>>> > > > > > > > >> > it's
>>> > > > > > > > >> > > > > > possible
>>> > > > > > > > >> > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > an
>>> > > > > > > > >> > > > > > > > > > > > > > > EndTxnRequest to be delayed
>>> on
>>> > the
>>> > > > > > > > >> coordinator.
>>> > > > > > > > >> > By
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > time
>>> > > > > > > > >> > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > delayed
>>> > > > > > > > >> > > > > > > > > > > > > > > EndTxnRequest is processed,
>>> it's
>>> > > > > > possible
>>> > > > > > > > that
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > > > previous
>>> > > > > > > > >> > > > > > > > txn
>>> > > > > > > > >> > > > > > > > > > has
>>> > > > > > > > >> > > > > > > > > > > > > > already
>>> > > > > > > > >> > > > > > > > > > > > > > > completed and a new txn has
>>> > > started.
>>> > > > > > > > >> Currently,
>>> > > > > > > > >> > > since
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > epoch
>>> > > > > > > > >> > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > not
>>> > > > > > > > >> > > > > > > > > > > > > > > bumped on every txn, the
>>> delayed
>>> > > > > > > > EndTxnRequest
>>> > > > > > > > >> > will
>>> > > > > > > > >> > > > add
>>> > > > > > > > >> > > > > > an
>>> > > > > > > > >> > > > > > > > > > > unexpected
>>> > > > > > > > >> > > > > > > > > > > > > > > prepare marker (and
>>> eventually a
>>> > > > > > complete
>>> > > > > > > > >> marker)
>>> > > > > > > > >> > > to
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > > ongoing
>>> > > > > > > > >> > > > > > > > > > > txn.
>>> > > > > > > > >> > > > > > > > > > > > > > This
>>> > > > > > > > >> > > > > > > > > > > > > > > won't cause the transaction
>>> to
>>> > > hang,
>>> > > > > but
>>> > > > > > > it
>>> > > > > > > > >> will
>>> > > > > > > > >> > > > break
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > EoS
>>> > > > > > > > >> > > > > > > > > > > > > semantic.
>>> > > > > > > > >> > > > > > > > > > > > > > > The proposal in this KIP
>>> will
>>> > > > address
>>> > > > > > this
>>> > > > > > > > >> issue
>>> > > > > > > > >> > > too.
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > 101. "However, I was
>>> writing it
>>> > so
>>> > > > > that
>>> > > > > > we
>>> > > > > > > > can
>>> > > > > > > > >> > > > > > distinguish
>>> > > > > > > > >> > > > > > > > > > between
>>> > > > > > > > >> > > > > > > > > > > > > > > old clients where we don't
>>> have
>>> > > the
>>> > > > > > > ability
>>> > > > > > > > do
>>> > > > > > > > >> > this
>>> > > > > > > > >> > > > > > > operation
>>> > > > > > > > >> > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > new
>>> > > > > > > > >> > > > > > > > > > > > > > > clients that can. (Old
>>> clients
>>> > > don't
>>> > > > > > bump
>>> > > > > > > > the
>>> > > > > > > > >> > epoch
>>> > > > > > > > >> > > > on
>>> > > > > > > > >> > > > > > > > commit,
>>> > > > > > > > >> > > > > > > > > so
>>> > > > > > > > >> > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > can't
>>> > > > > > > > >> > > > > > > > > > > > > > > say for sure the write
>>> belongs
>>> > to
>>> > > > the
>>> > > > > > > given
>>> > > > > > > > >> > > > > > transaction)."
>>> > > > > > > > >> > > > > > > > > > > > > > > 101.1 I am wondering why we
>>> need
>>> > > to
>>> > > > > > > > >> distinguish
>>> > > > > > > > >> > > > whether
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > > > marker
>>> > > > > > > > >> > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > written by the old and the
>>> new
>>> > > > client.
>>> > > > > > > Could
>>> > > > > > > > >> you
>>> > > > > > > > >> > > > > describe
>>> > > > > > > > >> > > > > > > > what
>>> > > > > > > > >> > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > do
>>> > > > > > > > >> > > > > > > > > > > > > > > differently if we know the
>>> > marker
>>> > > is
>>> > > > > > > written
>>> > > > > > > > >> by
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > new
>>> > > > > > > > >> > > > > > > > client?
>>> > > > > > > > >> > > > > > > > > > > > > > > 101.2 If we do need a way to
>>> > > > > distinguish
>>> > > > > > > > >> whether
>>> > > > > > > > >> > > the
>>> > > > > > > > >> > > > > > marker
>>> > > > > > > > >> > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > written
>>> > > > > > > > >> > > > > > > > > > > > > by
>>> > > > > > > > >> > > > > > > > > > > > > > > the old and the new client.
>>> > Would
>>> > > it
>>> > > > > be
>>> > > > > > > > >> simpler
>>> > > > > > > > >> > to
>>> > > > > > > > >> > > > just
>>> > > > > > > > >> > > > > > > > > > introduce a
>>> > > > > > > > >> > > > > > > > > > > > > > boolean
>>> > > > > > > > >> > > > > > > > > > > > > > > field instead of indirectly
>>> > > through
>>> > > > > the
>>> > > > > > > > >> previous
>>> > > > > > > > >> > > > > produce
>>> > > > > > > > >> > > > > > ID
>>> > > > > > > > >> > > > > > > > > > field?
>>> > > > > > > > >> > > > > > > > > > > > > > > 101.3 It's not clear to me
>>> why
>>> > we
>>> > > > only
>>> > > > > > add
>>> > > > > > > > the
>>> > > > > > > > >> > > > previous
>>> > > > > > > > >> > > > > > > > produce
>>> > > > > > > > >> > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > > field
>>> > > > > > > > >> > > > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > > > the complete marker, but
>>> not in
>>> > > the
>>> > > > > > > prepare
>>> > > > > > > > >> > marker.
>>> > > > > > > > >> > > > If
>>> > > > > > > > >> > > > > we
>>> > > > > > > > >> > > > > > > > want
>>> > > > > > > > >> > > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > know
>>> > > > > > > > >> > > > > > > > > > > > > > > whether a marker is written
>>> by
>>> > the
>>> > > > new
>>> > > > > > > > client
>>> > > > > > > > >> or
>>> > > > > > > > >> > > not,
>>> > > > > > > > >> > > > > it
>>> > > > > > > > >> > > > > > > > seems
>>> > > > > > > > >> > > > > > > > > > that
>>> > > > > > > > >> > > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > want
>>> > > > > > > > >> > > > > > > > > > > > > > > to do this consistently for
>>> all
>>> > > > > markers.
>>> > > > > > > > >> > > > > > > > > > > > > > > 101.4 What about the
>>> > > > > TransactionLogValue
>>> > > > > > > > >> record
>>> > > > > > > > >> > > > > > > representing
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > ongoing
>>> > > > > > > > >> > > > > > > > > > > > > > > state? Should we also
>>> > distinguish
>>> > > > > > whether
>>> > > > > > > > it's
>>> > > > > > > > >> > > > written
>>> > > > > > > > >> > > > > by
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > old
>>> > > > > > > > >> > > > > > > > > > > or
>>> > > > > > > > >> > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > new client?
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > 102. In the overflow case,
>>> it's
>>> > > > still
>>> > > > > > not
>>> > > > > > > > >> clear
>>> > > > > > > > >> > to
>>> > > > > > > > >> > > me
>>> > > > > > > > >> > > > > why
>>> > > > > > > > >> > > > > > > we
>>> > > > > > > > >> > > > > > > > > > write
>>> > > > > > > > >> > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > previous produce Id in the
>>> > prepare
>>> > > > > > marker
>>> > > > > > > > >> while
>>> > > > > > > > >> > > > writing
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > > next
>>> > > > > > > > >> > > > > > > > > > > > > produce
>>> > > > > > > > >> > > > > > > > > > > > > > Id
>>> > > > > > > > >> > > > > > > > > > > > > > > in the complete marker. You
>>> > > > mentioned
>>> > > > > > that
>>> > > > > > > > >> it's
>>> > > > > > > > >> > for
>>> > > > > > > > >> > > > > > > > > downgrading.
>>> > > > > > > > >> > > > > > > > > > > > > However,
>>> > > > > > > > >> > > > > > > > > > > > > > > we could downgrade with
>>> either
>>> > the
>>> > > > > > prepare
>>> > > > > > > > >> marker
>>> > > > > > > > >> > > or
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > > complete
>>> > > > > > > > >> > > > > > > > > > > > > marker.
>>> > > > > > > > >> > > > > > > > > > > > > > > In either case, the
>>> downgraded
>>> > > > > > coordinator
>>> > > > > > > > >> should
>>> > > > > > > > >> > > see
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > same
>>> > > > > > > > >> > > > > > > > > > > > produce
>>> > > > > > > > >> > > > > > > > > > > > > id
>>> > > > > > > > >> > > > > > > > > > > > > > > (probably the previous
>>> produce
>>> > > Id),
>>> > > > > > right?
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > Jun
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > On Wed, Dec 20, 2023 at
>>> 6:00 PM
>>> > > > > Justine
>>> > > > > > > > Olshan
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> <jo...@confluent.io.invalid>
>>> > > > > > > > >> > > > > > > > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > Hey Jun,
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > Thanks for taking a look
>>> at
>>> > the
>>> > > > KIP
>>> > > > > > > again.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > 100. For the epoch
>>> overflow
>>> > > case,
>>> > > > > only
>>> > > > > > > the
>>> > > > > > > > >> > marker
>>> > > > > > > > >> > > > > will
>>> > > > > > > > >> > > > > > > have
>>> > > > > > > > >> > > > > > > > > max
>>> > > > > > > > >> > > > > > > > > > > > > epoch.
>>> > > > > > > > >> > > > > > > > > > > > > > > This
>>> > > > > > > > >> > > > > > > > > > > > > > > > keeps the behavior of the
>>> rest
>>> > > of
>>> > > > > the
>>> > > > > > > > >> markers
>>> > > > > > > > >> > > where
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > last
>>> > > > > > > > >> > > > > > > > > > > marker
>>> > > > > > > > >> > > > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > epoch of the transaction
>>> > > records +
>>> > > > > 1.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > 101. You are correct that
>>> we
>>> > > don't
>>> > > > > > need
>>> > > > > > > to
>>> > > > > > > > >> > write
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > > since
>>> > > > > > > > >> > > > > > > > > > > > > > it
>>> > > > > > > > >> > > > > > > > > > > > > > > > is the same. However, I
>>> was
>>> > > > writing
>>> > > > > it
>>> > > > > > > so
>>> > > > > > > > >> that
>>> > > > > > > > >> > we
>>> > > > > > > > >> > > > can
>>> > > > > > > > >> > > > > > > > > > distinguish
>>> > > > > > > > >> > > > > > > > > > > > > > between
>>> > > > > > > > >> > > > > > > > > > > > > > > > old clients where we don't
>>> > have
>>> > > > the
>>> > > > > > > > ability
>>> > > > > > > > >> do
>>> > > > > > > > >> > > this
>>> > > > > > > > >> > > > > > > > operation
>>> > > > > > > > >> > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > new
>>> > > > > > > > >> > > > > > > > > > > > > > > > clients that can. (Old
>>> clients
>>> > > > don't
>>> > > > > > > bump
>>> > > > > > > > >> the
>>> > > > > > > > >> > > epoch
>>> > > > > > > > >> > > > > on
>>> > > > > > > > >> > > > > > > > > commit,
>>> > > > > > > > >> > > > > > > > > > so
>>> > > > > > > > >> > > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > > can't
>>> > > > > > > > >> > > > > > > > > > > > > > > > say for sure the write
>>> belongs
>>> > > to
>>> > > > > the
>>> > > > > > > > given
>>> > > > > > > > >> > > > > > transaction).
>>> > > > > > > > >> > > > > > > > If
>>> > > > > > > > >> > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > receive
>>> > > > > > > > >> > > > > > > > > > > > > > > an
>>> > > > > > > > >> > > > > > > > > > > > > > > > EndTxn request from a new
>>> > > client,
>>> > > > we
>>> > > > > > > will
>>> > > > > > > > >> fill
>>> > > > > > > > >> > > this
>>> > > > > > > > >> > > > > > > field.
>>> > > > > > > > >> > > > > > > > We
>>> > > > > > > > >> > > > > > > > > > can
>>> > > > > > > > >> > > > > > > > > > > > > > > guarantee
>>> > > > > > > > >> > > > > > > > > > > > > > > > that any EndTxn requests
>>> with
>>> > > the
>>> > > > > same
>>> > > > > > > > epoch
>>> > > > > > > > >> > are
>>> > > > > > > > >> > > > from
>>> > > > > > > > >> > > > > > the
>>> > > > > > > > >> > > > > > > > > same
>>> > > > > > > > >> > > > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > > > > the same transaction.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > 102. In prepare phase, we
>>> have
>>> > > the
>>> > > > > > same
>>> > > > > > > > >> > producer
>>> > > > > > > > >> > > ID
>>> > > > > > > > >> > > > > and
>>> > > > > > > > >> > > > > > > > epoch
>>> > > > > > > > >> > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > always
>>> > > > > > > > >> > > > > > > > > > > > > > > > had. It is the producer
>>> ID and
>>> > > > epoch
>>> > > > > > > that
>>> > > > > > > > >> are
>>> > > > > > > > >> > on
>>> > > > > > > > >> > > > the
>>> > > > > > > > >> > > > > > > > marker.
>>> > > > > > > > >> > > > > > > > > In
>>> > > > > > > > >> > > > > > > > > > > > > commit
>>> > > > > > > > >> > > > > > > > > > > > > > > > phase, we stay the same
>>> unless
>>> > > it
>>> > > > is
>>> > > > > > the
>>> > > > > > > > >> > overflow
>>> > > > > > > > >> > > > > case.
>>> > > > > > > > >> > > > > > > In
>>> > > > > > > > >> > > > > > > > > that
>>> > > > > > > > >> > > > > > > > > > > > case,
>>> > > > > > > > >> > > > > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > > > set the producer ID to
>>> the new
>>> > > one
>>> > > > > we
>>> > > > > > > > >> generated
>>> > > > > > > > >> > > and
>>> > > > > > > > >> > > > > > epoch
>>> > > > > > > > >> > > > > > > > to
>>> > > > > > > > >> > > > > > > > > 0
>>> > > > > > > > >> > > > > > > > > > > > after
>>> > > > > > > > >> > > > > > > > > > > > > > > > complete. This is for
>>> > downgrade
>>> > > > > > > > >> compatibility.
>>> > > > > > > > >> > > The
>>> > > > > > > > >> > > > > > tagged
>>> > > > > > > > >> > > > > > > > > > fields
>>> > > > > > > > >> > > > > > > > > > > > are
>>> > > > > > > > >> > > > > > > > > > > > > > just
>>> > > > > > > > >> > > > > > > > > > > > > > > > safety guards for retries
>>> and
>>> > > > > > failovers.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > In prepare phase for epoch
>>> > > > overflow
>>> > > > > > case
>>> > > > > > > > >> only
>>> > > > > > > > >> > we
>>> > > > > > > > >> > > > > store
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > next
>>> > > > > > > > >> > > > > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > > > ID. This is for the case
>>> where
>>> > > we
>>> > > > > > reload
>>> > > > > > > > the
>>> > > > > > > > >> > > > > > transaction
>>> > > > > > > > >> > > > > > > > > > > > coordinator
>>> > > > > > > > >> > > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > > > > prepare state. Once the
>>> > > > transaction
>>> > > > > is
>>> > > > > > > > >> > committed,
>>> > > > > > > > >> > > > we
>>> > > > > > > > >> > > > > > can
>>> > > > > > > > >> > > > > > > > use
>>> > > > > > > > >> > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > > > ID the client already is
>>> > using.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > In commit phase, we store
>>> the
>>> > > > > previous
>>> > > > > > > > >> producer
>>> > > > > > > > >> > > ID
>>> > > > > > > > >> > > > in
>>> > > > > > > > >> > > > > > > case
>>> > > > > > > > >> > > > > > > > of
>>> > > > > > > > >> > > > > > > > > > > > > retries.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > I think it is easier to
>>> think
>>> > of
>>> > > > it
>>> > > > > as
>>> > > > > > > > just
>>> > > > > > > > >> how
>>> > > > > > > > >> > > we
>>> > > > > > > > >> > > > > were
>>> > > > > > > > >> > > > > > > > > storing
>>> > > > > > > > >> > > > > > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > > > > > and epoch before, with
>>> some
>>> > > extra
>>> > > > > > > > bookeeping
>>> > > > > > > > >> > and
>>> > > > > > > > >> > > > edge
>>> > > > > > > > >> > > > > > > case
>>> > > > > > > > >> > > > > > > > > > > handling
>>> > > > > > > > >> > > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > tagged fields. We have to
>>> do
>>> > it
>>> > > > this
>>> > > > > > way
>>> > > > > > > > for
>>> > > > > > > > >> > > > > > > compatibility
>>> > > > > > > > >> > > > > > > > > with
>>> > > > > > > > >> > > > > > > > > > > > > > > downgrades.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > 103. Next producer ID is
>>> for
>>> > > > prepare
>>> > > > > > > > status
>>> > > > > > > > >> and
>>> > > > > > > > >> > > > > > previous
>>> > > > > > > > >> > > > > > > > > > producer
>>> > > > > > > > >> > > > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > > > > > > > after complete. The
>>> reason why
>>> > > we
>>> > > > > need
>>> > > > > > > two
>>> > > > > > > > >> > > separate
>>> > > > > > > > >> > > > > > > > (tagged)
>>> > > > > > > > >> > > > > > > > > > > fields
>>> > > > > > > > >> > > > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > > > > > > > backwards compatibility.
>>> We
>>> > need
>>> > > > to
>>> > > > > > keep
>>> > > > > > > > the
>>> > > > > > > > >> > same
>>> > > > > > > > >> > > > > > > semantics
>>> > > > > > > > >> > > > > > > > > for
>>> > > > > > > > >> > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > non-tagged field in case
>>> we
>>> > > > > downgrade.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > 104. We set the fields as
>>> we
>>> > do
>>> > > in
>>> > > > > the
>>> > > > > > > > >> > > > transactional
>>> > > > > > > > >> > > > > > > state
>>> > > > > > > > >> > > > > > > > > (as
>>> > > > > > > > >> > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > need
>>> > > > > > > > >> > > > > > > > > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > > > > > do this for compatibility
>>> --
>>> > if
>>> > > we
>>> > > > > > > > >> downgrade,
>>> > > > > > > > >> > we
>>> > > > > > > > >> > > > will
>>> > > > > > > > >> > > > > > > only
>>> > > > > > > > >> > > > > > > > > have
>>> > > > > > > > >> > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > non-tagged fields) It
>>> will be
>>> > > the
>>> > > > > old
>>> > > > > > > > >> producer
>>> > > > > > > > >> > ID
>>> > > > > > > > >> > > > and
>>> > > > > > > > >> > > > > > max
>>> > > > > > > > >> > > > > > > > > > epoch.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > Hope this helps. Let me
>>> know
>>> > if
>>> > > > you
>>> > > > > > have
>>> > > > > > > > >> > further
>>> > > > > > > > >> > > > > > > questions.
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > Justine
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > On Wed, Dec 20, 2023 at
>>> > 3:33 PM
>>> > > > Jun
>>> > > > > > Rao
>>> > > > > > > > >> > > > > > > > > > <jun@confluent.io.invalid
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > wrote:
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > Hi, Justine,
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > It seems that you have
>>> made
>>> > > some
>>> > > > > > > changes
>>> > > > > > > > >> to
>>> > > > > > > > >> > > > KIP-890
>>> > > > > > > > >> > > > > > > since
>>> > > > > > > > >> > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > vote.
>>> > > > > > > > >> > > > > > > > > > > > > > In
>>> > > > > > > > >> > > > > > > > > > > > > > > > > particular, we are
>>> changing
>>> > > the
>>> > > > > > format
>>> > > > > > > > of
>>> > > > > > > > >> > > > > > > > > > TransactionLogValue.
>>> > > > > > > > >> > > > > > > > > > > A
>>> > > > > > > > >> > > > > > > > > > > > > few
>>> > > > > > > > >> > > > > > > > > > > > > > > > > comments related to
>>> that.
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > 100. Just to be clear.
>>> The
>>> > > > > overflow
>>> > > > > > > case
>>> > > > > > > > >> > (i.e.
>>> > > > > > > > >> > > > > when a
>>> > > > > > > > >> > > > > > > new
>>> > > > > > > > >> > > > > > > > > > > > > producerId
>>> > > > > > > > >> > > > > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > > > generated) is when the
>>> > current
>>> > > > > epoch
>>> > > > > > > > >> equals
>>> > > > > > > > >> > to
>>> > > > > > > > >> > > > max
>>> > > > > > > > >> > > > > -
>>> > > > > > > > >> > > > > > 1
>>> > > > > > > > >> > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > not
>>> > > > > > > > >> > > > > > > > > > > > max?
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > 101. For the "not epoch
>>> > > > overflow"
>>> > > > > > > case,
>>> > > > > > > > we
>>> > > > > > > > >> > > write
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > > previous
>>> > > > > > > > >> > > > > > > > > > > ID
>>> > > > > > > > >> > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > > tagged field in the
>>> complete
>>> > > > > phase.
>>> > > > > > Do
>>> > > > > > > > we
>>> > > > > > > > >> > need
>>> > > > > > > > >> > > to
>>> > > > > > > > >> > > > > do
>>> > > > > > > > >> > > > > > > that
>>> > > > > > > > >> > > > > > > > > > since
>>> > > > > > > > >> > > > > > > > > > > > > > produce
>>> > > > > > > > >> > > > > > > > > > > > > > > > id
>>> > > > > > > > >> > > > > > > > > > > > > > > > > doesn't change in this
>>> case?
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > 102. It seems that the
>>> > meaning
>>> > > > for
>>> > > > > > the
>>> > > > > > > > >> > > > > > > > > > ProducerId/ProducerEpoch
>>> > > > > > > > >> > > > > > > > > > > > > > fields
>>> > > > > > > > >> > > > > > > > > > > > > > > in
>>> > > > > > > > >> > > > > > > > > > > > > > > > > TransactionLogValue
>>> changes
>>> > > > > > depending
>>> > > > > > > on
>>> > > > > > > > >> the
>>> > > > > > > > >> > > > > > > > > > TransactionStatus.
>>> > > > > > > > >> > > > > > > > > > > > > When
>>> > > > > > > > >> > > > > > > > > > > > > > > > > the TransactionStatus is
>>> > > > ongoing,
>>> > > > > > they
>>> > > > > > > > >> > > represent
>>> > > > > > > > >> > > > > the
>>> > > > > > > > >> > > > > > > > > current
>>> > > > > > > > >> > > > > > > > > > > > > > ProducerId
>>> > > > > > > > >> > > > > > > > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > > > > > the current
>>> ProducerEpoch.
>>> > > When
>>> > > > > the
>>> > > > > > > > >> > > > > TransactionStatus
>>> > > > > > > > >> > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> PrepareCommit/PrepareAbort,
>>> > > they
>>> > > > > > > > represent
>>> > > > > > > > >> > the
>>> > > > > > > > >> > > > > > current
>>> > > > > > > > >> > > > > > > > > > > ProducerId
>>> > > > > > > > >> > > > > > > > > > > > > and
>>> > > > > > > > >> > > > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > > next ProducerEpoch.
>>> When the
>>> > > > > > > > >> > TransactionStatus
>>> > > > > > > > >> > > is
>>> > > > > > > > >> > > > > > > > > > Commit/Abort,
>>> > > > > > > > >> > > > > > > > > > > > > they
>>> > > > > > > > >> > > > > > > > > > > > > > > > > further depend on
>>> whether
>>> > the
>>> > > > > epoch
>>> > > > > > > > >> overflows
>>> > > > > > > > >> > > or
>>> > > > > > > > >> > > > > not.
>>> > > > > > > > >> > > > > > > If
>>> > > > > > > > >> > > > > > > > > > there
>>> > > > > > > > >> > > > > > > > > > > is
>>> > > > > > > > >> > > > > > > > > > > > > no
>>> > > > > > > > >> > > > > > > > > > > > > > > > > overflow, they represent
>>> > the
>>> > > > > > current
>>> > > > > > > > >> > > ProducerId
>>> > > > > > > > >> > > > > and
>>> > > > > > > > >> > > > > > > the
>>> > > > > > > > >> > > > > > > > > next
>>> > > > > > > > >> > > > > > > > > > > > > > > > ProducerEpoch
>>> > > > > > > > >> > > > > > > > > > > > > > > > > (max). Otherwise, they
>>> > > represent
>>> > > > > the
>>> > > > > > > > newly
>>> > > > > > > > >> > > > > generated
>>> > > > > > > > >> > > > > > > > > > ProducerId
>>> > > > > > > > >> > > > > > > > > > > > > and a
>>> > > > > > > > >> > > > > > > > > > > > > > > > > ProducerEpoch of 0. Is
>>> that
>>> > > > right?
>>> > > > > > > This
>>> > > > > > > > >> seems
>>> > > > > > > > >> > > not
>>> > > > > > > > >> > > > > > easy
>>> > > > > > > > >> > > > > > > to
>>> > > > > > > > >> > > > > > > > > > > > > understand.
>>> > > > > > > > >> > > > > > > > > > > > > > > > Could
>>> > > > > > > > >> > > > > > > > > > > > > > > > > we provide some examples
>>> > like
>>> > > > what
>>> > > > > > > Artem
>>> > > > > > > > >> has
>>> > > > > > > > >> > > done
>>> > > > > > > > >> > > > > in
>>> > > > > > > > >> > > > > > > > > KIP-939?
>>> > > > > > > > >> > > > > > > > > > > > Have
>>> > > > > > > > >> > > > > > > > > > > > > we
>>> > > > > > > > >> > > > > > > > > > > > > > > > > considered a simpler
>>> design
>>> > > > where
>>> > > > > > > > >> > > > > > > > ProducerId/ProducerEpoch
>>> > > > > > > > >> > > > > > > > > > > always
>>> > > > > > > > >> > > > > > > > > > > > > > > > represent
>>> > > > > > > > >> > > > > > > > > > > > > > > > > the same value (e.g.
>>> for the
>>> > > > > current
>>> > > > > > > > >> > > transaction)
>>> > > > > > > > >> > > > > > > > > independent
>>> > > > > > > > >> > > > > > > > > > > of
>>> > > > > > > > >> > > > > > > > > > > > > the
>>> > > > > > > > >> > > > > > > > > > > > > > > > > TransactionStatus and
>>> epoch
>>> > > > > > overflow?
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > 103. It's not clear to
>>> me
>>> > why
>>> > > we
>>> > > > > > need
>>> > > > > > > 3
>>> > > > > > > > >> > fields:
>>> > > > > > > > >> > > > > > > > ProducerId,
>>> > > > > > > > >> > > > > > > > > > > > > > > > PrevProducerId,
>>> > > > > > > > >> > > > > > > > > > > > > > > > > NextProducerId. Could we
>>> > just
>>> > > > have
>>> > > > > > > > >> ProducerId
>>> > > > > > > > >> > > and
>>> > > > > > > > >> > > > > > > > > > > NextProducerId?
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > 104. For
>>> > > WriteTxnMarkerRequests,
>>> > > > > if
>>> > > > > > > the
>>> > > > > > > > >> > > producer
>>> > > > > > > > >> > > > > > epoch
>>> > > > > > > > >> > > > > > > > > > > overflows,
>>> > > > > > > > >> > > > > > > > > > > > > > what
>>> > > > > > > > >> > > > > > > > > > > > > > > do
>>> > > > > > > > >> > > > > > > > > > > > > > > > > we set the producerId
>>> and
>>> > the
>>> > > > > > > > >> producerEpoch?
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > Thanks,
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > > > Jun
>>> > > > > > > > >> > > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > > >
>>> > > > > > > > >> > > > > > > > > >
>>> > > > > > > > >> > > > > > > > >
>>> > > > > > > > >> > > > > > > >
>>> > > > > > > > >> > > > > > >
>>> > > > > > > > >> > > > > >
>>> > > > > > > > >> > > > >
>>> > > > > > > > >> > > >
>>> > > > > > > > >> > >
>>> > > > > > > > >> >
>>> > > > > > > > >>
>>> > > > > > > > >
>>> > > > > > > >
>>> > > > > > >
>>> > > > > >
>>> > > > >
>>> > > >
>>> > >
>>> >
>>>
>>